sort by:
Revision Author Date Message Commit Date
b668811 Preparing Spark release v3.0.0-preview2-rc1 16 December 2019, 12:14:54 UTC
ba0f59b [SPARK-30265][INFRA] Do not change R version when releasing preview versions ### What changes were proposed in this pull request? This PR makes it do not change R version when releasing preview versions. ### Why are the changes needed? Failed to make Spark binary distribution: ``` ++ . /opt/spark-rm/output/spark-3.0.0-preview2-bin-hadoop2.7/R/find-r.sh +++ '[' -z /usr/bin ']' ++ /usr/bin/Rscript -e ' if("devtools" %in% rownames(installed.packages())) { library(devtools); devtools::document(pkg="./pkg", roclets=c("rd")) }' Loading required package: usethis Updating SparkR documentation First time using roxygen2. Upgrading automatically... Loading SparkR Invalid DESCRIPTION: Malformed package version. See section 'The DESCRIPTION file' in the 'Writing R Extensions' manual. Error: invalid version specification '3.0.0-preview2' In addition: Warning message: roxygen2 requires Encoding: UTF-8 Execution halted [ERROR] Command execution failed. org.apache.commons.exec.ExecuteException: Process exited with an error: 1 (Exit value: 1) at org.apache.commons.exec.DefaultExecutor.executeInternal (DefaultExecutor.java:404) at org.apache.commons.exec.DefaultExecutor.execute (DefaultExecutor.java:166) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:804) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:751) at org.codehaus.mojo.exec.ExecMojo.execute (ExecMojo.java:313) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo (DefaultBuildPluginManager.java:137) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:210) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:156) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:148) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:117) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:81) at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build (SingleThreadedBuilder.java:56) at org.apache.maven.lifecycle.internal.LifecycleStarter.execute (LifecycleStarter.java:128) at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:305) at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:192) at org.apache.maven.DefaultMaven.execute (DefaultMaven.java:105) at org.apache.maven.cli.MavenCli.execute (MavenCli.java:957) at org.apache.maven.cli.MavenCli.doMain (MavenCli.java:289) at org.apache.maven.cli.MavenCli.main (MavenCli.java:193) at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke (Method.java:498) at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced (Launcher.java:282) at org.codehaus.plexus.classworlds.launcher.Launcher.launch (Launcher.java:225) at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode (Launcher.java:406) at org.codehaus.plexus.classworlds.launcher.Launcher.main (Launcher.java:347) [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-preview2: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 18.619 s] [INFO] Spark Project Tags ................................. SUCCESS [ 13.652 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 5.673 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 2.081 s] [INFO] Spark Project Networking ........................... SUCCESS [ 3.509 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 0.993 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 7.556 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 5.522 s] [INFO] Spark Project Core ................................. FAILURE [01:06 min] [INFO] Spark Project ML Local Library ..................... SKIPPED [INFO] Spark Project GraphX ............................... SKIPPED [INFO] Spark Project Streaming ............................ SKIPPED [INFO] Spark Project Catalyst ............................. SKIPPED [INFO] Spark Project SQL .................................. SKIPPED [INFO] Spark Project ML Library ........................... SKIPPED [INFO] Spark Project Tools ................................ SKIPPED [INFO] Spark Project Hive ................................. SKIPPED [INFO] Spark Project Graph API ............................ SKIPPED [INFO] Spark Project Cypher ............................... SKIPPED [INFO] Spark Project Graph ................................ SKIPPED [INFO] Spark Project REPL ................................. SKIPPED [INFO] Spark Project Assembly ............................. SKIPPED [INFO] Kafka 0.10+ Token Provider for Streaming ........... SKIPPED [INFO] Spark Integration for Kafka 0.10 ................... SKIPPED [INFO] Kafka 0.10+ Source for Structured Streaming ........ SKIPPED [INFO] Spark Project Examples ............................. SKIPPED [INFO] Spark Integration for Kafka 0.10 Assembly .......... SKIPPED [INFO] Spark Avro ......................................... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 02:04 min [INFO] Finished at: 2019-12-16T08:02:45Z [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal org.codehaus.mojo:exec-maven-plugin:1.6.0:exec (sparkr-pkg) on project spark-core_2.12: Command execution failed.: Process exited with an error: 1 (Exit value: 1) -> [Help 1] [ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch. [ERROR] Re-run Maven using the -X switch to enable full debug logging. [ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles: [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException [ERROR] [ERROR] After correcting the problems, you can resume the build with the command [ERROR] mvn <args> -rf :spark-core_2.12 ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: ```diff diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cdb59093781..b648c51e010 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R -336,8 +336,8 sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") - # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) + # Remove -preview2 from jvm versions + jvmVersionStrip <- gsub("-preview2", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { ``` Closes #26904 from wangyum/SPARK-30265. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Yuming Wang <wgyumg@gmail.com> 16 December 2019, 11:54:12 UTC
fdcd0e7 [SPARK-30192][SQL] support column position in DS v2 ### What changes were proposed in this pull request? update DS v2 API to support add/alter column with column position ### Why are the changes needed? We have a parser rule for column position, but we fail the query if it's specified, because the builtin catalog can't support add/alter column with column position. Since we have the catalog plugin API now, we should let the catalog implementation to decide if it supports column position or not. ### Does this PR introduce any user-facing change? not yet ### How was this patch tested? new tests Closes #26817 from cloud-fan/parser. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 16 December 2019, 10:55:17 UTC
72f5597 [SPARK-30104][SQL][FOLLOWUP] Remove LookupCatalog.AsTemporaryViewIdentifier ### What changes were proposed in this pull request? As discussed in https://github.com/apache/spark/pull/26741#discussion_r357504518, `LookupCatalog.AsTemporaryViewIdentifier` is no longer used and can be removed. ### Why are the changes needed? Code clean up ### Does this PR introduce any user-facing change? No ### How was this patch tested? Removed tests that were testing solely `AsTemporaryViewIdentifier` extractor. Closes #26897 from imback82/30104-followup. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 16 December 2019, 09:43:01 UTC
3bf5498 [MINOR][DOCS] Fix documentation for slide function ### What changes were proposed in this pull request? This PR proposes to fix documentation for slide function. Fixed the spacing issue and added some parameter related info. ### Why are the changes needed? Documentation improvement ### Does this PR introduce any user-facing change? No (doc-only change). ### How was this patch tested? Manually tested by documentation build. Closes #26896 from bboutkov/pyspark_doc_fix. Authored-by: Boris Boutkov <boris.boutkov@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 16 December 2019, 07:29:09 UTC
1fc353d Revert "[SPARK-30056][INFRA] Skip building test artifacts in `dev/make-distribution.sh` ### What changes were proposed in this pull request? This reverts commit 7c0ce285. ### Why are the changes needed? Failed to make distribution: ``` [INFO] -----------------< org.apache.spark:spark-sketch_2.12 >----------------- [INFO] Building Spark Project Sketch 3.0.0-preview2 [3/33] [INFO] --------------------------------[ jar ]--------------------------------- [INFO] Downloading from central: https://repo.maven.apache.org/maven2/org/apache/spark/spark-tags_2.12/3.0.0-preview2/spark-tags_2.12-3.0.0-preview2-tests.jar [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-preview2: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 26.513 s] [INFO] Spark Project Tags ................................. SUCCESS [ 48.393 s] [INFO] Spark Project Sketch ............................... FAILURE [ 0.034 s] [INFO] Spark Project Local DB ............................. SKIPPED [INFO] Spark Project Networking ........................... SKIPPED [INFO] Spark Project Shuffle Streaming Service ............ SKIPPED [INFO] Spark Project Unsafe ............................... SKIPPED [INFO] Spark Project Launcher ............................. SKIPPED [INFO] Spark Project Core ................................. SKIPPED [INFO] Spark Project ML Local Library ..................... SKIPPED [INFO] Spark Project GraphX ............................... SKIPPED [INFO] Spark Project Streaming ............................ SKIPPED [INFO] Spark Project Catalyst ............................. SKIPPED [INFO] Spark Project SQL .................................. SKIPPED [INFO] Spark Project ML Library ........................... SKIPPED [INFO] Spark Project Tools ................................ SKIPPED [INFO] Spark Project Hive ................................. SKIPPED [INFO] Spark Project Graph API ............................ SKIPPED [INFO] Spark Project Cypher ............................... SKIPPED [INFO] Spark Project Graph ................................ SKIPPED [INFO] Spark Project REPL ................................. SKIPPED [INFO] Spark Project YARN Shuffle Service ................. SKIPPED [INFO] Spark Project YARN ................................. SKIPPED [INFO] Spark Project Mesos ................................ SKIPPED [INFO] Spark Project Kubernetes ........................... SKIPPED [INFO] Spark Project Hive Thrift Server ................... SKIPPED [INFO] Spark Project Assembly ............................. SKIPPED [INFO] Kafka 0.10+ Token Provider for Streaming ........... SKIPPED [INFO] Spark Integration for Kafka 0.10 ................... SKIPPED [INFO] Kafka 0.10+ Source for Structured Streaming ........ SKIPPED [INFO] Spark Project Examples ............................. SKIPPED [INFO] Spark Integration for Kafka 0.10 Assembly .......... SKIPPED [INFO] Spark Avro ......................................... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 01:15 min [INFO] Finished at: 2019-12-16T05:29:43Z [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal on project spark-sketch_2.12: Could not resolve dependencies for project org.apache.spark:spark-sketch_2.12:jar:3.0.0-preview2: Could not find artifact org.apache.spark:spark-tags_2.12:jar:tests:3.0.0-preview2 in central (https://repo.maven.apache.org/maven2) -> [Help 1] [ERROR] ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test. Closes #26902 from wangyum/SPARK-30056. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Yuming Wang <wgyumg@gmail.com> 16 December 2019, 06:16:17 UTC
0a2afce [SPARK-30200][SQL][FOLLOW-UP] Expose only explain(mode: String) in Scala side, and clean up related codes ### What changes were proposed in this pull request? This PR mainly targets: 1. Expose only explain(mode: String) in Scala side 2. Clean up related codes - Hide `ExplainMode` under private `execution` package. No particular reason but just because `ExplainUtils` exists there - Use `case object` + `trait` pattern in `ExplainMode` to look after `ParseMode`. - Move `Dataset.toExplainString` to `QueryExecution.explainString` to look after `QueryExecution.simpleString`, and deduplicate the codes at `ExplainCommand`. - Use `ExplainMode` in `ExplainCommand` too. - Add `explainString` to `PythonSQLUtils` to avoid unexpected test failure of PySpark during refactoring Scala codes side. ### Why are the changes needed? To minimised exposed APIs, deduplicate, and clean up. ### Does this PR introduce any user-facing change? `Dataset.explain(mode: ExplainMode)` will be removed (which only exists in master). ### How was this patch tested? Manually tested and existing tests should cover. Closes #26898 from HyukjinKwon/SPARK-30200-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 16 December 2019, 05:42:35 UTC
26b658f [SPARK-30253][INFRA] Do not add commits when releasing preview version ### What changes were proposed in this pull request? This PR add support do not add commits to master branch when releasing preview version. ### Why are the changes needed? We need manual revert this change, example: ![image](https://user-images.githubusercontent.com/5399861/70788945-f9d15180-1dcc-11ea-81f5-c0d89c28440a.png) ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test Closes #26879 from wangyum/SPARK-30253. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Yuming Wang <wgyumg@gmail.com> 16 December 2019, 02:44:29 UTC
67b644c [SPARK-30166][SQL] Eliminate compilation warnings in JSONOptions ### What changes were proposed in this pull request? In the PR, I propose to replace `setJacksonOptions()` in `JSONOptions` by `buildJsonFactory()` which builds `JsonFactory` using `JsonFactoryBuilder`. This allows to avoid using **deprecated** feature configurations from `JsonParser.Feature`. ### Why are the changes needed? - The changes eliminate the following compilation warnings in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala`: ``` Warning:Warning:line (137)Java enum ALLOW_NUMERIC_LEADING_ZEROS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_NUMERIC_LEADING_ZEROS, allowNumericLeadingZeros) Warning:Warning:line (138)Java enum ALLOW_NON_NUMERIC_NUMBERS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers) Warning:Warning:line (139)Java enum ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER, Warning:Warning:line (141)Java enum ALLOW_UNQUOTED_CONTROL_CHARS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars) ``` - This put together building JsonFactory and set options from JSONOptions. So, we will not forget to call `setJacksonOptions` in the future. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By `JsonSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`. Closes #26797 from MaxGekk/eliminate-warning. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Sean Owen <srowen@gmail.com> 15 December 2019, 14:45:57 UTC
58b2939 [SPARK-30173][INFRA] Automatically close stale PRs ### What changes were proposed in this pull request? This PR adds [a GitHub workflow to automatically close stale PRs](https://github.com/marketplace/actions/close-stale-issues). ### Why are the changes needed? This will help cut down the number of open but stale PRs and keep the PR queue manageable. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I'm not sure how to test this PR without impacting real PRs on the repo. See: https://github.com/actions/stale/issues/32 Closes #26877 from nchammas/SPARK-30173-stale-prs. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: Sean Owen <srowen@gmail.com> 15 December 2019, 14:42:16 UTC
a9fbd31 [SPARK-30240][CORE] Support HTTP redirects directly to a proxy server ### What changes were proposed in this pull request? The PR adds a new config option to configure an address for the proxy server, and a new handler that intercepts redirects and replaces the URL with one pointing at the proxy server. This is needed on top of the "proxy base path" support because redirects use full URLs, not just absolute paths from the server's root. ### Why are the changes needed? Spark's web UI has support for generating links to paths with a prefix, to support a proxy server, but those do not apply when the UI is responding with redirects. In that case, Spark is sending its own URL back to the client, and if it's behind a dumb proxy server that doesn't do rewriting (like when using stunnel for HTTPS support) then the client will see the wrong URL and may fail. ### Does this PR introduce any user-facing change? Yes. It's a new UI option. ### How was this patch tested? Tested with added unit test, with Spark behind stunnel, and in a more complicated app using a different HTTPS proxy. Closes #26873 from vanzin/SPARK-30240. Authored-by: Marcelo Vanzin <vanzin@cloudera.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 15 December 2019, 01:39:06 UTC
fb2f5a4 [SPARK-25100][CORE] Register TaskCommitMessage to KyroSerializer ## What changes were proposed in this pull request? Fix the bug when invoking saveAsNewAPIHadoopDataset to store data, the job will fail because the class TaskCommitMessage hasn't be registered if serializer is KryoSerializer and spark.kryo.registrationRequired is true ## How was this patch tested? UT Closes #26714 from deshanxiao/SPARK-25100. Authored-by: xiaodeshan <xiaodeshan@xiaomi.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 15 December 2019, 01:15:30 UTC
4cbef89 [SPARK-30259][SQL] Fix CREATE TABLE behavior when session catalog is specified explicitly ### What changes were proposed in this pull request? Fix bug : CREATE TABLE throw error when session catalog specified explicitly. ### Why are the changes needed? Currently, Spark throw error when the session catalog is specified explicitly in "CREATE TABLE" and "CREATE TABLE AS SELECT" command, eg.  > CREATE TABLE spark_catalog.tbl USING json AS SELECT 1 AS i; the error message is like below: > 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_table : db=spark_catalog tbl=tbl > 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr cmd=get_table : db=spark_catalog tbl=tbl > 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_database: spark_catalog > 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr cmd=get_database: spark_catalog > 19/12/14 10:56:08 WARN ObjectStore: Failed to get database spark_catalog, returning NoSuchObjectException > Error in query: Database 'spark_catalog' not found; ### Does this PR introduce any user-facing change? Yes, after this PR, "CREATE TALBE" and "CREATE TABLE AS SELECT" can complete successfully when session catalog "spark_catalog" specified explicitly. ### How was this patch tested? New unit tests added. Closes #26887 from fuwhu/SPARK-30259. Authored-by: fuwhu <bestwwg@163.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 14 December 2019, 23:36:14 UTC
f483a13 [SPARK-30231][SQL][PYTHON][FOLLOWUP] Make error messages clear in PySpark df.explain ### What changes were proposed in this pull request? This pr is a followup of #26861 to address minor comments from viirya. ### Why are the changes needed? For better error messages. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested. Closes #26886 from maropu/SPARK-30231-FOLLOWUP. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 14 December 2019, 22:26:50 UTC
46e950b [SPARK-30263][CORE] Don't log potentially sensitive value of non-Spark properties ignored in spark-submit ### What changes were proposed in this pull request? The value of non-Spark config properties ignored in spark-submit is no longer logged. ### Why are the changes needed? The value isn't really needed in the logs, and could contain potentially sensitive info. While we can redact the values selectively too, I figured it's more robust to just not log them at all here, as the values aren't important in this log statement. ### Does this PR introduce any user-facing change? Other than the change to logging above, no. ### How was this patch tested? Existing tests Closes #26893 from srowen/SPARK-30263. Authored-by: Sean Owen <srowen@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 14 December 2019, 21:13:54 UTC
d3ec8b1 [SPARK-30066][SQL] Support columnar execution on interval types ### What changes were proposed in this pull request? Columnar execution support for interval types ### Why are the changes needed? support cache tables with interval columns improve performance too ### Does this PR introduce any user-facing change? Yes cache table with accept interval columns ### How was this patch tested? add ut Closes #26699 from yaooqinn/SPARK-30066. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 14 December 2019, 21:10:46 UTC
f197204 [SPARK-30236][SQL][DOCS] Clarify date and time patterns supported in docs ### What changes were proposed in this pull request? Link to appropriate Java Class with list of date/time patterns supported ### Why are the changes needed? Avoid confusion on the end-user's side of things, as seen in questions like [this](https://stackoverflow.com/questions/54496878/date-format-conversion-is-adding-1-year-to-the-border-dates) on StackOverflow ### Does this PR introduce any user-facing change? Yes, Docs are updated. ### How was this patch tested? `date_format`: ![image](https://user-images.githubusercontent.com/2394761/70796647-b5c55900-1d9a-11ea-89f9-7a8661641c09.png) `to_unix_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796664-c07fee00-1d9a-11ea-9029-e82d899e3f59.png) `unix_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796688-caa1ec80-1d9a-11ea-8868-a18c437a5d49.png) `from_unixtime`: ![image](https://user-images.githubusercontent.com/2394761/70796703-d4c3eb00-1d9a-11ea-85fe-3c672e0cda28.png) `to_date`: ![image](https://user-images.githubusercontent.com/2394761/70796718-dd1c2600-1d9a-11ea-81f4-a0966eeb0f1d.png) `to_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796735-e6a58e00-1d9a-11ea-8ef7-d3e1d9b5370f.png) Closes #26864 from johnhany97/SPARK-30236. Authored-by: John Ayad <johnhany97@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 14 December 2019, 21:08:15 UTC
4c37a8a [SPARK-30143][SS] Add a timeout on stopping a streaming query ### What changes were proposed in this pull request? Add a timeout configuration for StreamingQuery.stop() ### Why are the changes needed? The stop() method on a Streaming Query awaits the termination of the stream execution thread. However, the stream execution thread may block forever depending on the streaming source implementation (like in Kafka, which runs UninterruptibleThreads). This causes control flow applications to hang indefinitely as well. We'd like to introduce a timeout to stop the execution thread, so that the control flow thread can decide to do an action if a timeout is hit. ### Does this PR introduce any user-facing change? By default, no. If the timeout configuration is set, then a TimeoutException will be thrown if a stream cannot be stopped within the given timeout. ### How was this patch tested? Unit tests Closes #26771 from brkyvz/stopTimeout. Lead-authored-by: Burak Yavuz <brkyvz@gmail.com> Co-authored-by: Burak Yavuz <burak@databricks.com> Signed-off-by: Burak Yavuz <brkyvz@gmail.com> 13 December 2019, 23:16:00 UTC
61ebc81 [SPARK-30167][REPL] Log4j configuration for REPL can't override the root logger properly ### What changes were proposed in this pull request? In the current implementation of `SparkShellLoggingFilter`, if the log level of the root logger and the log level of a message are different, whether a message should logged is decided based on log4j's configuration but whether the message should be output to the REPL's console is not cared. So, if the log level of the root logger is `DEBUG`, the log level of REPL's logger is `WARN` and the log level of a message is `INFO`, the message will output to the REPL's console even though `INFO < WARN`. https://github.com/apache/spark/pull/26798/files#diff-bfd5810d8aa78ad90150e806d830bb78L237 The ideal behavior should be like as follows and implemented them in this change. 1. If the log level of a message is greater than or equal to the log level of the root logger, the message should be logged but whether the message is output to the REPL's console should be decided based on whether the log level of the message is greater than or equal to the log level of the REPL's logger. 2. If a log level or custom appenders are explicitly defined for a category, whether a log message via the logger corresponding to the category is logged and output to the REPL's console should be decided baed on the log level of the category. We can confirm whether a log level or appenders are explicitly set to a logger for a category by `Logger#getLevel` and `Logger#getAllAppenders.hasMoreElements`. ### Why are the changes needed? This is a bug breaking a compatibility. #9816 enabled REPL's log4j configuration to override root logger but #23675 seems to have broken the feature. You can see one example when you modifies the default log4j configuration like as follows. ``` # Change the log level for rootCategory to DEBUG log4j.rootCategory=DEBUG, console ... # The log level for repl.Main remains WARN log4j.logger.org.apache.spark.repl.Main=WARN ``` If you launch REPL with the configuration, INFO level logs appear even though the log level for REPL is WARN. ``` ・・・ 19/12/08 23:31:38 INFO Utils: Successfully started service 'sparkDriver' on port 33083. 19/12/08 23:31:38 INFO SparkEnv: Registering MapOutputTracker 19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMaster 19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information 19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up 19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMasterHeartbeat ・・・ ``` Before #23675 was applied, those INFO level logs are not shown with the same log4j.properties. ### Does this PR introduce any user-facing change? Yes. The logging behavior for REPL is fixed. ### How was this patch tested? Manual test and newly added unit test. Closes #26798 from sarutak/fix-spark-shell-loglevel. Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com> 13 December 2019, 22:30:11 UTC
ec26dde [SPARK-29455][WEBUI] Improve tooltip information for Stages ### What changes were proposed in this pull request? Adding tooltip to Stages tab for better usability. ### Why are the changes needed? There are a few common points of confusion in the UI that could be clarified with tooltips. We should add tooltips to explain. ### Does this PR introduce any user-facing change? Yes ![image](https://user-images.githubusercontent.com/29914590/70693889-5a389400-1ce4-11ea-91bb-ee1e997a5c35.png) ### How was this patch tested? Manual Closes #26859 from sharangk/tooltip1. Authored-by: sharan.gk <sharan.gk@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 13 December 2019, 19:35:00 UTC
e1ee3fb [SPARK-30216][INFRA] Use python3 in Docker release image ### What changes were proposed in this pull request? - Reverts commit 1f94bf4 and d6be46e - Switches python to python3 in Docker release image. ### Why are the changes needed? `dev/make-distribution.sh` and `python/setup.py` are use python3. https://github.com/apache/spark/pull/26844/files#diff-ba2c046d92a1d2b5b417788bfb5cb5f8L236 https://github.com/apache/spark/pull/26330/files#diff-8cf6167d58ce775a08acafcfe6f40966 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: ``` yumwangubuntu-3513086:~/spark$ dev/create-release/do-release-docker.sh -n -d /home/yumwang/spark-release Output directory already exists. Overwrite and continue? [y/n] y Branch [branch-2.4]: master Current branch version is 3.0.0-SNAPSHOT. Release [3.0.0]: 3.0.0-preview2 RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [master]: ASF user [yumwang]: Full name [Yuming Wang]: GPG key [yumwangapache.org]: DBD447010C1B4F7DAD3F7DFD6E1B4122F6A3A338 ================ Release details: BRANCH: master VERSION: 3.0.0-preview2 TAG: v3.0.0-preview2-rc1 NEXT: 3.0.1-SNAPSHOT ASF USER: yumwang GPG KEY: DBD447010C1B4F7DAD3F7DFD6E1B4122F6A3A338 FULL NAME: Yuming Wang E-MAIL: yumwangapache.org ================ Is this info correct [y/n]? y GPG passphrase: ======================== = Building spark-rm image with tag latest... Command: docker build -t spark-rm:latest --build-arg UID=110302528 /home/yumwang/spark/dev/create-release/spark-rm Log file: docker-build.log Building v3.0.0-preview2-rc1; output will be at /home/yumwang/spark-release/output gpg: directory '/home/spark-rm/.gnupg' created gpg: keybox '/home/spark-rm/.gnupg/pubring.kbx' created gpg: /home/spark-rm/.gnupg/trustdb.gpg: trustdb created gpg: key 6E1B4122F6A3A338: public key "Yuming Wang <yumwangapache.org>" imported gpg: key 6E1B4122F6A3A338: secret key imported gpg: Total number processed: 1 gpg: imported: 1 gpg: secret keys read: 1 gpg: secret keys imported: 1 ======================== = Creating release tag v3.0.0-preview2-rc1... Command: /opt/spark-rm/release-tag.sh Log file: tag.log It may take some time for the tag to be synchronized to github. Press enter when you've verified that the new tag (v3.0.0-preview2-rc1) is available. ======================== = Building Spark... Command: /opt/spark-rm/release-build.sh package Log file: build.log ======================== = Building documentation... Command: /opt/spark-rm/release-build.sh docs Log file: docs.log ======================== = Publishing release Command: /opt/spark-rm/release-build.sh publish-release Log file: publish.log ``` Generated doc: ![image](https://user-images.githubusercontent.com/5399861/70693075-a7723100-1cf7-11ea-9f88-9356a02349a1.png) Closes #26848 from wangyum/SPARK-30216. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 13 December 2019, 19:31:31 UTC
4da9780 Revert "[SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%'" This reverts commit cada5beef72530fa699b5ec13d67261be37730e4. Closes #26883 from gengliangwang/revert. Authored-by: Gengliang Wang <gengliang.wang@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 13 December 2019, 19:23:55 UTC
cc276f8 [SPARK-30243][BUILD][K8S] Upgrade K8s client dependency to 4.6.4 ### What changes were proposed in this pull request? This PR aims to upgrade K8s client library from 4.6.1 to 4.6.4 for `3.0.0-preview2`. ### Why are the changes needed? This will bring the latest bug fixes. - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.4 - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.3 - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.2 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with K8s integration test. Closes #26874 from dongjoon-hyun/SPARK-30243. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 13 December 2019, 16:25:51 UTC
ac9b188 [SPARK-30248][SQL] Fix DROP TABLE behavior when session catalog name is provided in the identifier ### What changes were proposed in this pull request? If a table name is qualified with session catalog name `spark_catalog`, the `DROP TABLE` command fails. For example, the following ``` sql("CREATE TABLE tbl USING json AS SELECT 1 AS i") sql("DROP TABLE spark_catalog.tbl") ``` fails with: ``` org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'spark_catalog' not found; at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists(ExternalCatalog.scala:42) at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists$(ExternalCatalog.scala:40) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.requireDbExists(InMemoryCatalog.scala:45) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.tableExists(InMemoryCatalog.scala:336) ``` This PR correctly resolves `spark_catalog` as a catalog. ### Why are the changes needed? It's fixing a bug. ### Does this PR introduce any user-facing change? Yes, now, the `spark_catalog.tbl` in the above example is dropped as expected. ### How was this patch tested? Added a test. Closes #26878 from imback82/fix_drop_table. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 13 December 2019, 13:45:35 UTC
64c7b94 [SPARK-30231][SQL][PYTHON] Support explain mode in PySpark df.explain ### What changes were proposed in this pull request? This pr intends to support explain modes implemented in #26829 for PySpark. ### Why are the changes needed? For better debugging info. in PySpark dataframes. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UTs. Closes #26861 from maropu/ExplainModeInPython. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 13 December 2019, 08:44:23 UTC
94eb665 [SPARK-30227][SQL] Add close() on DataWriter interface ### What changes were proposed in this pull request? This patch adds close() method to the DataWriter interface, which will become the place to cleanup the resource. ### Why are the changes needed? The lifecycle of DataWriter instance ends at either commit() or abort(). That makes datasource implementors to feel they can place resource cleanup in both sides, but abort() can be called when commit() fails; so they have to ensure they don't do double-cleanup if cleanup is not idempotent. ### Does this PR introduce any user-facing change? Depends on the definition of user; if they're developers of custom DSv2 source, they have to add close() in their DataWriter implementations. It's OK to just add close() with empty content as they should have already dealt with resource cleanup in commit/abort, but they would love to migrate the resource cleanup logic to close() as it avoids double cleanup. If they're just end users using the provided DSv2 source (regardless of built-in/3rd party), no change. ### How was this patch tested? Existing tests. Closes #26855 from HeartSaVioR/SPARK-30227. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 13 December 2019, 08:12:41 UTC
cb6d2b3 [SPARK-30040][SQL] DROP FUNCTION should do multi-catalog resolution ### What changes were proposed in this pull request? Add DropFunctionStatement and make DROP FUNCTION go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing DROP FUNCTION namespace.function ### Does this PR introduce any user-facing change? Yes. When running DROP FUNCTION namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26854 from planga82/feature/SPARK-30040_DropFunctionV2Catalog. Authored-by: Pablo Langa <soypab@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 12 December 2019, 23:15:54 UTC
5114389 [SPARK-30107][SQL] Expose nested schema pruning to all V2 sources ### What changes were proposed in this pull request? This PR exposes the existing logic for nested schema pruning to all sources, which is in line with the description of `SupportsPushDownRequiredColumns` . Right now, `SchemaPruning` (rule, not helper utility) is applied in the optimizer directly on certain instances of `Table` ignoring `SupportsPushDownRequiredColumns` that is part of `ScanBuilder`. I think it would be cleaner to perform schema pruning and filter push-down in one place. Therefore, this PR moves all the logic into `V2ScanRelationPushDown`. ### Why are the changes needed? This change allows all V2 data sources to benefit from nested column pruning (if they support it). ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR mostly relies on existing tests. On top, it adds one test to verify that top-level schema pruning works as well as one test for predicates with subqueries. Closes #26751 from aokolnychyi/nested-schema-pruning-ds-v2. Authored-by: Anton Okolnychyi <aokolnychyi@apple.com> Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com> 12 December 2019, 21:40:46 UTC
982f72f [SPARK-30238][SQL] hive partition pruning can only support string and integral types ### What changes were proposed in this pull request? Check the partition column data type and only allow string and integral types in hive partition pruning. ### Why are the changes needed? Currently we only support string and integral types in hive partition pruning, but the check is done for literals. If the predicate is `InSet`, then there is no literal and we may pass an unsupported partition predicate to Hive and cause problems. ### Does this PR introduce any user-facing change? yes. fix a bug. A query fails before and can run now. ### How was this patch tested? a new test Closes #26871 from cloud-fan/bug. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 12 December 2019, 21:07:20 UTC
cada5be [SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%' ### What changes were proposed in this pull request? Since [25001](https://github.com/apache/spark/pull/25001), spark support like escape syntax. But '%' and '_' is the reserve char in `Like` expression. We can not use them as escape char. ### Why are the changes needed? Avoid some unexpect problem when using like escape syntax. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Add UT. Closes #26860 from ulysses-you/SPARK-30230. Authored-by: ulysses <youxiduo@weidian.com> Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com> 12 December 2019, 17:52:27 UTC
39c0696 [MINOR] Fix google style guide address ### What changes were proposed in this pull request? This PR update google style guide address to `https://google.github.io/styleguide/javaguide.html`. ### Why are the changes needed? `https://google-styleguide.googlecode.com/svn-history/r130/trunk/javaguide.html` **404**: ![image](https://user-images.githubusercontent.com/5399861/70717915-431c9500-1d2a-11ea-895b-024be953a116.png) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Closes #26865 from wangyum/fix-google-styleguide. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Sean Owen <srowen@gmail.com> 12 December 2019, 17:04:01 UTC
cc087a3 [SPARK-30162][SQL] Add PushedFilters to metadata in Parquet DSv2 implementation ### What changes were proposed in this pull request? This PR proposes to add `PushedFilters` into metadata to show the pushed filters in Parquet DSv2 implementation. In case of ORC, it is already added at https://github.com/apache/spark/pull/24719/files#diff-0fc82694b20da3cd2cbb07206920eef7R62-R64 ### Why are the changes needed? In order for users to be able to debug, and to match with ORC. ### Does this PR introduce any user-facing change? ```scala spark.range(10).write.mode("overwrite").parquet("/tmp/foo") spark.read.parquet("/tmp/foo").filter("5 > id").explain() ``` **Before:** ``` == Physical Plan == *(1) Project [id#20L] +- *(1) Filter (isnotnull(id#20L) AND (5 > id#20L)) +- *(1) ColumnarToRow +- BatchScan[id#20L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct<id:bigint> ``` **After:** ``` == Physical Plan == *(1) Project [id#13L] +- *(1) Filter (isnotnull(id#13L) AND (5 > id#13L)) +- *(1) ColumnarToRow +- BatchScan[id#13L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct<id:bigint>, PushedFilters: [IsNotNull(id), LessThan(id,5)] ``` ### How was this patch tested? Unittest were added and manually tested. Closes #26857 from HyukjinKwon/SPARK-30162. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 12 December 2019, 16:33:33 UTC
fd39b6d [SQL] Typo in HashedRelation error ### What changes were proposed in this pull request? Fixed typo in exception message of HashedRelations ### Why are the changes needed? Better exception messages ### Does this PR introduce any user-facing change? No ### How was this patch tested? No tests needed Closes #26822 from aaron-lau/master. Authored-by: Aaron Lau <aaron.lau@datadoghq.com> Signed-off-by: Sean Owen <srowen@gmail.com> 12 December 2019, 14:42:18 UTC
25de90e [SPARK-30170][SQL][MLLIB][TESTS] Eliminate compilation warnings: part 1 ### What changes were proposed in this pull request? - Replace `Seq[String]` by `Seq[_]` in `StopWordsRemoverSuite` because `String` type is unchecked due erasure. - Throw an exception for default case in `MLTest.checkNominalOnDF` because we don't expect other attribute types currently. - Explicitly cast float to double in `BigDecimal(y)`. This is what the `apply()` method does for `float`s. - Replace deprecated `verifyZeroInteractions` by `verifyNoInteractions`. - Equivalent replacement of `\0` by `\u0000` in `CSVExprUtilsSuite` - Import `scala.language.implicitConversions` in `CollectionExpressionsSuite`, `HashExpressionsSuite` and in `ExpressionParserSuite`. ### Why are the changes needed? The changes fix compiler warnings showed in the JIRA ticket https://issues.apache.org/jira/browse/SPARK-30170 . Eliminating the warning highlights other warnings which could take more attention to real problems. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites `StopWordsRemoverSuite`, `AnalysisExternalCatalogSuite`, `CSVExprUtilsSuite`, `CollectionExpressionsSuite`, `HashExpressionsSuite`, `ExpressionParserSuite` and sub-tests of `MLTest`. Closes #26799 from MaxGekk/eliminate-warning-2. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Sean Owen <srowen@gmail.com> 12 December 2019, 14:38:15 UTC
ce61ee8 [SPARK-30126][CORE] support space in file path and name for addFile and addJar function ### What changes were proposed in this pull request? sparkContext.addFile and sparkContext.addJar fails when file path contains spaces ### Why are the changes needed? When uploading a file to the spark context via the addFile and addJar function, an exception is thrown when file path contains a space character. Escaping the space with %20 or or + doesn't change the result. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add test case. Closes #26773 from 07ARB/SPARK-30126. Authored-by: 07ARB <ankitrajboudh@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 December 2019, 12:30:47 UTC
8e9bfea [SPARK-29188][PYTHON] toPandas (without Arrow) gets wrong dtypes when applied on empty DF ### What changes were proposed in this pull request? An empty Spark DataFrame converted to a Pandas DataFrame wouldn't have the right column types. Several type mappings were missing. ### Why are the changes needed? Empty Spark DataFrames can be used to write unit tests, and verified by converting them to Pandas first. But this can fail when the column types are wrong. ### Does this PR introduce any user-facing change? Yes; the error reported in the JIRA issue should not happen anymore. ### How was this patch tested? Through unit tests in `pyspark.sql.tests.test_dataframe.DataFrameTests#test_to_pandas_from_empty_dataframe` Closes #26747 from dlindelof/SPARK-29188. Authored-by: David <dlindelof@expediagroup.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 12 December 2019, 11:49:10 UTC
2936507 [SPARK-30150][SQL] ADD FILE, ADD JAR, LIST FILE & LIST JAR Command do not accept quoted path ### What changes were proposed in this pull request? `add file "abc.txt"` and `add file 'abc.txt'` are not supported. For these two spark sql gives `FileNotFoundException`. Only `add file abc.txt` is supported currently. After these changes path can be given as quoted text for ADD FILE, ADD JAR, LIST FILE, LIST JAR commands in spark-sql ### Why are the changes needed? In many of the spark-sql commands (like create table ,etc )we write path in quoted format only. To maintain this consistency we should support quoted format with this command as well. ### Does this PR introduce any user-facing change? Yes. Now users can write path with quotes. ### How was this patch tested? Manually tested. Closes #26779 from iRakson/SPARK-30150. Authored-by: root1 <raksonrakesh@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 December 2019, 09:11:21 UTC
3741a36 [SPARK-30104][SQL][FOLLOWUP] V2 catalog named 'global_temp' should always be masked ### What changes were proposed in this pull request? This is a follow up to #26741 to address the following: 1. V2 catalog named `global_temp` should always be masked. 2. #26741 introduces `CatalogAndIdentifer` that supersedes `CatalogObjectIdentfier`. This PR removes `CatalogObjectIdentfier` and its usages and replace them with `CatalogAndIdentifer`. 3. `CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog)` and `CatalogObjectIdentifier(catalog, ident) if isSessionCatalog(catalog)` are replaced with `NonSessionCatalogAndIdentifier` and `SessionCatalogAndIdentifier` respectively. ### Why are the changes needed? To fix an existing with handling v2 catalog named `global_temp` and to simplify the code base. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added new tests. Closes #26853 from imback82/lookup_table. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 December 2019, 06:47:20 UTC
b709091 [SPARK-30228][BUILD] Update zstd-jni to 1.4.4-3 ### What changes were proposed in this pull request? This PR aims to update zstd-jni library to 1.4.4-3. ### Why are the changes needed? This will bring the latest bug fixes in zstd itself and some performance improvement. - https://github.com/facebook/zstd/releases/tag/v1.4.4 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26856 from dongjoon-hyun/SPARK-ZSTD-144. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 12 December 2019, 05:16:32 UTC
1ced6c1 [SPARK-30213][SQL] Remove the mutable status in ShuffleQueryStageExec ### What changes were proposed in this pull request? Currently `ShuffleQueryStageExec `contain the mutable status, eg `mapOutputStatisticsFuture `variable. So It is not easy to pass when we copy `ShuffleQueryStageExec`. This PR will put the `mapOutputStatisticsFuture ` variable from `ShuffleQueryStageExec` to `ShuffleExchangeExec`. And then we can pass the value of `mapOutputStatisticsFuture ` when copying. ### Why are the changes needed? In order to remove the mutable status in `ShuffleQueryStageExec` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing uts Closes #26846 from JkSelf/removeMutableVariable. Authored-by: jiake <ke.a.jia@intel.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 12 December 2019, 03:39:31 UTC
40b9c89 [SPARK-30199][DSTREAM] Recover `spark.(ui|blockManager).port` from checkpoint ### What changes were proposed in this pull request? This PR aims to recover `spark.ui.port` and `spark.blockManager.port` from checkpoint like `spark.driver.port`. ### Why are the changes needed? When the user configures these values, we can respect them. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the newly added test cases. Closes #26827 from dongjoon-hyun/SPARK-30199. Authored-by: Aaruna <aaruna@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 12 December 2019, 02:21:36 UTC
b4aeaf9 [SPARK-30198][CORE] BytesToBytesMap does not grow internal long array as expected ### What changes were proposed in this pull request? This patch changes the condition to check if BytesToBytesMap should grow up its internal array. Specifically, it changes to compare by the capacity of the array, instead of its size. ### Why are the changes needed? One Spark job on our cluster hangs forever at BytesToBytesMap.safeLookup. After inspecting, the long array size is 536870912. Currently in BytesToBytesMap.append, we only grow the internal array if the size of the array is less than its MAX_CAPACITY that is 536870912. So in above case, the array can not be grown up, and safeLookup can not find an empty slot forever. But it is wrong because we use two array entries per key, so the array size is twice the capacity. We should compare the current capacity of the array, instead of its size. ### Does this PR introduce any user-facing change? No ### How was this patch tested? This issue only happens when loading big number of values into BytesToBytesMap, so it is hard to do unit test. This is tested manually with internal Spark job. Closes #26828 from viirya/fix-bytemap. Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com> Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 22:58:21 UTC
9cf9304 [SPARK-30038][SQL] DESCRIBE FUNCTION should do multi-catalog resolution ### What changes were proposed in this pull request? Add DescribeFunctionsStatement and make DESCRIBE FUNCTIONS go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing DESCRIBE FUNCTIONS namespace.function ### Does this PR introduce any user-facing change? Yes. When running DESCRIBE FUNCTIONS namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26840 from planga82/feature/SPARK-30038_DescribeFunction_V2Catalog. Authored-by: Pablo Langa <soypab@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 22:02:58 UTC
33f53cb [SPARK-30195][SQL][CORE][ML] Change some function, import definitions to work with stricter compiler in Scala 2.13 ### What changes were proposed in this pull request? See https://issues.apache.org/jira/browse/SPARK-30195 for the background; I won't repeat it here. This is sort of a grab-bag of related issues. ### Why are the changes needed? To cross-compile with Scala 2.13 later. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests for 2.12. I've been manually checking that this actually resolves the compile problems in 2.13 separately. Closes #26826 from srowen/SPARK-30195. Authored-by: Sean Owen <srowen@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 20:33:58 UTC
e39bb4c [MINOR][SS][DOC] Fix the ss-kafka doc for availability of 'minPartitions' option ### What changes were proposed in this pull request? This patch fixes the availability of `minPartitions` option for Kafka source, as it is only supported by micro-batch for now. There's a WIP PR for batch (#25436) as well but there's no progress on the PR so far, so safer to fix the doc first, and let it be added later when we address it with batch case as well. ### Why are the changes needed? The doc is wrong and misleading. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Just a doc change. Closes #26849 from HeartSaVioR/MINOR-FIX-minPartition-availability-doc. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 17:23:39 UTC
e933539 [SPARK-29864][SPARK-29920][SQL] Strict parsing of day-time strings to intervals ### What changes were proposed in this pull request? In the PR, I propose new implementation of `fromDayTimeString` which strictly parses strings in day-time formats to intervals. New implementation accepts only strings that match to a pattern defined by the `from` and `to`. Here is the mapping of user's bounds and patterns: - `[+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS]` for **DAY TO SECOND** - `[+|-]D+ H[H]:m[m]` for **DAY TO MINUTE** - `[+|-]D+ H[H]` for **DAY TO HOUR** - `[+|-]H[H]:m[m]s[s][.SSSSSSSSS]` for **HOUR TO SECOND** - `[+|-]H[H]:m[m]` for **HOUR TO MINUTE** - `[+|-]m[m]:s[s][.SSSSSSSSS]` for **MINUTE TO SECOND** Closes #26327 Closes #26358 ### Why are the changes needed? - Improve user experience with Spark SQL, and respect to the bound specified by users. - Behave the same as other broadly used DBMS - Oracle and MySQL. ### Does this PR introduce any user-facing change? Yes, before: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; interval 1 weeks 3 days 11 hours 12 minutes ``` After: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; Error in query: requirement failed: Interval string must match day-time format of '^(?<sign>[+|-])?(?<hour>\d{1,2}):(?<minute>\d{1,2})$': 10 11:12:13.123(line 1, pos 16) == SQL == SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE ----------------^^^ ``` ### How was this patch tested? - Added tests to `IntervalUtilsSuite` - By `ExpressionParserSuite` - Updated `literals.sql` Closes #26473 from MaxGekk/strict-from-daytime-string. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 11 December 2019, 17:08:53 UTC
a59cb13 [SPARK-30200][SQL][FOLLOWUP] Fix typo in ExplainMode ### What changes were proposed in this pull request? This pr is a follow-up of #26829 to fix typos in ExplainMode. ### Why are the changes needed? For better docs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26851 from maropu/SPARK-30200-FOLLOWUP. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 16:17:53 UTC
d46c03c [SPARK-29460][WEBUI] Add tooltip for Jobs page ### What changes were proposed in this pull request? Adding tooltip for jobs tab column - Job Id (Job Group), Description ,Submitted, Duration, Stages, Tasks Before: ![Screenshot from 2019-11-04 11-31-02](https://user-images.githubusercontent.com/51401130/68102467-e8a54300-fef8-11e9-9f9e-48dd1b393ac8.png) After: ![Screenshot from 2019-11-04 11-30-53](https://user-images.githubusercontent.com/51401130/68102478-f3f86e80-fef8-11e9-921a-357678229cb4.png) ### Why are the changes needed? Jobs tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual Closes #26384 from PavithraRamachandran/jobTab_tooltip. Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com> Signed-off-by: Sean Owen <srowen@gmail.com> 11 December 2019, 15:39:39 UTC
82418b4 [SPARK-30207][SQL][DOCS] Enhance the SQL NULL Semantics document ### What changes were proposed in this pull request? Enhancement of the SQL NULL Semantics document: sql-ref-null-semantics.html. ### Why are the changes needed? Clarify the behavior of `UNKNOWN` for both `EXIST` and `IN` operation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Doc changes only. Closes #26837 from xuanyuanking/SPARK-30207. Authored-by: Yuanjian Li <xyliyuanjian@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 11 December 2019, 12:41:07 UTC
99ea324 [SPARK-27506][SQL] Allow deserialization of Avro data using compatible schemas Follow up of https://github.com/apache/spark/pull/24405 ### What changes were proposed in this pull request? The current implementation of _from_avro_ and _AvroDataToCatalyst_ doesn't allow doing schema evolution since it requires the deserialization of an Avro record with the exact same schema with which it was serialized. The proposed change is to add a new option `actualSchema` to allow passing the schema used to serialize the records. This allows using a different compatible schema for reading by passing both schemas to _GenericDatumReader_. If no writer's schema is provided, nothing changes from before. ### Why are the changes needed? Consider the following example. ``` // schema ID: 1 val schema1 = """ { "type": "record", "name": "MySchema", "fields": [ {"name": "col1", "type": "int"}, {"name": "col2", "type": "string"} ] } """ // schema ID: 2 val schema2 = """ { "type": "record", "name": "MySchema", "fields": [ {"name": "col1", "type": "int"}, {"name": "col2", "type": "string"}, {"name": "col3", "type": "string", "default": ""} ] } """ ``` The two schemas are compatible - i.e. you can use `schema2` to deserialize events serialized with `schema1`, in which case there will be the field `col3` with the default value. Now imagine that you have two dataframes (read from batch or streaming), one with Avro events from schema1 and the other with events from schema2. **We want to combine them into one dataframe** for storing or further processing. With the current `from_avro` function we can only decode each of them with the corresponding schema: ``` scalaval df1 = ... // Avro events created with schema1 df1: org.apache.spark.sql.DataFrame = [eventBytes: binary] scalaval decodedDf1 = df1.select(from_avro('eventBytes, schema1) as "decoded") decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string>] scalaval df2= ... // Avro events created with schema2 df2: org.apache.spark.sql.DataFrame = [eventBytes: binary] scalaval decodedDf2 = df2.select(from_avro('eventBytes, schema2) as "decoded") decodedDf2: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string, col3: string>] ``` but then `decodedDf1` and `decodedDf2` have different Spark schemas and we can't union them. Instead, with the proposed change we can decode `df1` in the following way: ``` scalaimport scala.collection.JavaConverters._ scalaval decodedDf1 = df1.select(from_avro(data = 'eventBytes, jsonFormatSchema = schema2, options = Map("actualSchema" -> schema1).asJava) as "decoded") decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string, col3: string>] ``` so that both dataframes have the same schemas and can be merged. ### Does this PR introduce any user-facing change? This PR allows users to pass a new configuration but it doesn't affect current code. ### How was this patch tested? A new unit test was added. Closes #26780 from Fokko/SPARK-27506. Lead-authored-by: Fokko Driesprong <fokko@apache.org> Co-authored-by: Gianluca Amori <gianluca.amori@gmail.com> Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com> 11 December 2019, 09:26:29 UTC
beae14d [SPARK-30104][SQL] Fix catalog resolution for 'global_temp' ### What changes were proposed in this pull request? `global_temp` is used as a database name to access global temp views. The current catalog lookup logic considers only the first element of multi-part name when it resolves a catalog. This results in using the session catalog even `global_temp` is used as a table name under v2 catalog. This PR addresses this by making sure multi-part name has two elements before using the session catalog. ### Why are the changes needed? Currently, 'global_temp' can be used as a table name in certain commands (CREATE) but not in others (DESCRIBE): ``` // Assume "spark.sql.globalTempDatabase" is set to "global_temp". sql(s"CREATE TABLE testcat.t (id bigint, data string) USING foo") sql(s"CREATE TABLE testcat.global_temp (id bigint, data string) USING foo") sql("USE testcat") sql(s"DESCRIBE TABLE t").show +---------------+---------+-------+ | col_name|data_type|comment| +---------------+---------+-------+ | id| bigint| | | data| string| | | | | | | # Partitioning| | | |Not partitioned| | | +---------------+---------+-------+ sql(s"DESCRIBE TABLE global_temp").show org.apache.spark.sql.AnalysisException: Table not found: global_temp;; 'DescribeTable 'UnresolvedV2Relation [global_temp], org.apache.spark.sql.connector.InMemoryTableSessionCatalog2f1af64f, `global_temp`, false at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:47) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:46) at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:122) ``` ### Does this PR introduce any user-facing change? Yes, `sql(s"DESCRIBE TABLE global_temp").show` in the above example now displays: ``` +---------------+---------+-------+ | col_name|data_type|comment| +---------------+---------+-------+ | id| bigint| | | data| string| | | | | | | # Partitioning| | | |Not partitioned| | | +---------------+---------+-------+ ``` instead of throwing an exception. ### How was this patch tested? Added new tests. Closes #26741 from imback82/global_temp. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 11 December 2019, 08:56:42 UTC
eb50996 [SPARK-30211][INFRA] Use python3 in make-distribution.sh ### What changes were proposed in this pull request? This PR switches python to python3 in `make-distribution.sh`. ### Why are the changes needed? SPARK-29672 changed this - https://github.com/apache/spark/pull/26330/files#diff-8cf6167d58ce775a08acafcfe6f40966 ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26844 from wangyum/SPARK-30211. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 07:30:12 UTC
3cc55f6 [SPARK-29392][CORE][SQL][FOLLOWUP] More removal of 'foo Symbol syntax for Scala 2.13 ### What changes were proposed in this pull request? Another continuation of https://github.com/apache/spark/pull/26748 ### Why are the changes needed? To cleanly cross compile with Scala 2.13. ### Does this PR introduce any user-facing change? None. ### How was this patch tested? Existing tests Closes #26842 from srowen/SPARK-29392.4. Authored-by: Sean Owen <srowen@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 11 December 2019, 03:41:24 UTC
d7843dd [SPARK-29152][CORE] Executor Plugin shutdown when dynamic allocation is enabled ### What changes were proposed in this pull request? Added `shutdownHook` for shutdown method of executor plugin. This will ensure that shutdown method will be called always. ### Why are the changes needed? Whenever executors are not going down gracefully, i.e getting killed due to idle time or getting killed forcefully, shutdown method of executors plugin is not getting called. Shutdown method can be used to release any resources that plugin has acquired during its initialisation. So its important to make sure that every time a executor goes down shutdown method of plugin gets called. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Tested Manually. Closes #26810 from iRakson/Executor_Plugin. Authored-by: root1 <raksonrakesh@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com> 10 December 2019, 22:23:51 UTC
cfd7ca9 Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer" This reverts commit 3641c3dd69b2bd2beae028d52356450cc41f69ed. 10 December 2019, 21:38:38 UTC
ad238a2 [SPARK-29976][CORE] Trigger speculation for stages with too few tasks ### What changes were proposed in this pull request? This PR add an optional spark conf for speculation to allow speculative runs for stages where there are only a few tasks. ``` spark.speculation.task.duration.threshold ``` If provided, tasks would be speculatively run if the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold. ### Why are the changes needed? This change helps avoid scenarios where there is single executor that could hang forever due to disk issue and we unfortunately assigned the single task in a TaskSet to that executor and cause the whole job to hang forever. ### Does this PR introduce any user-facing change? yes. If the new config `spark.speculation.task.duration.threshold` is provided and the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold, then speculative tasks would be submitted for the running tasks in the TaskSet. ### How was this patch tested? Unit tests are added to TaskSetManagerSuite. Closes #26614 from yuchenhuo/SPARK-29976. Authored-by: Yuchen Huo <yuchen.huo@databricks.com> Signed-off-by: Thomas Graves <tgraves@apache.org> 10 December 2019, 20:43:26 UTC
aec1d95 [SPARK-30205][PYSPARK] Import ABCs from collections.abc to remove deprecation warnings ### What changes were proposed in this pull request? This PR aims to remove deprecation warnings by importing ABCs from `collections.abc` instead of `collections`. - https://github.com/python/cpython/pull/10596 ### Why are the changes needed? This will remove deprecation warnings in Python 3.7 and 3.8. ``` $ python -V Python 3.7.5 $ python python/pyspark/resultiterable.py python/pyspark/resultiterable.py:23: DeprecationWarning: Using or importing the ABCs from 'collections' instead of from 'collections.abc' is deprecated since Python 3.3,and in 3.9 it will stop working class ResultIterable(collections.Iterable): ``` ### Does this PR introduce any user-facing change? No, this doesn't introduce user-facing change ### How was this patch tested? Manually because this is about deprecation warning messages. Closes #26835 from tirkarthi/spark-30205-fix-abc-warnings. Authored-by: Karthikeyan Singaravelan <tir.karthi@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 10 December 2019, 19:08:13 UTC
8f0eb7d [SPARK-29587][SQL] Support SQL Standard type real as float(4) numeric as decimal ### What changes were proposed in this pull request? The types decimal and numeric are equivalent. Both types are part of the SQL standard. the real type is 4 bytes, variable-precision, inexact, 6 decimal digits precision, same as our float, part of the SQL standard. ### Why are the changes needed? improve sql standard support other dbs https://www.postgresql.org/docs/9.3/datatype-numeric.html https://prestodb.io/docs/current/language/types.html#floating-point http://www.sqlservertutorial.net/sql-server-basics/sql-server-data-types/ MySQL treats REAL as a synonym for DOUBLE PRECISION (a nonstandard variation), unless the REAL_AS_FLOAT SQL mode is enabled. In MySQL, NUMERIC is implemented as DECIMAL, so the following remarks about DECIMAL apply equally to NUMERIC. ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26537 from yaooqinn/SPARK-29587. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 December 2019, 18:22:08 UTC
24c4ce1 [SPARK-28351][SQL][FOLLOWUP] Remove 'DELETE FROM' from unsupportedHiveNativeCommands ### What changes were proposed in this pull request? Minor change, rm `DELETE FROM` from unsupported hive native operation, because it is supported in parser. ### Why are the changes needed? clear ambiguous ambiguous ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #26836 from yaooqinn/SPARK-28351. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 10 December 2019, 17:54:50 UTC
6103cf1 [SPARK-30200][SQL] Add ExplainMode for Dataset.explain ### What changes were proposed in this pull request? This pr intends to add `ExplainMode` for explaining `Dataset/DataFrame` with a given format mode (`ExplainMode`). `ExplainMode` has four types along with the SQL EXPLAIN command: `Simple`, `Extended`, `Codegen`, `Cost`, and `Formatted`. For example, this pr enables users to explain DataFrame/Dataset with the `FORMATTED` format implemented in #24759; ``` scala> spark.range(10).groupBy("id").count().explain(ExplainMode.Formatted) == Physical Plan == * HashAggregate (3) +- * HashAggregate (2) +- * Range (1) (1) Range [codegen id : 1] Output: [id#0L] (2) HashAggregate [codegen id : 1] Input: [id#0L] (3) HashAggregate [codegen id : 1] Input: [id#0L, count#8L] ``` This comes from [the cloud-fan suggestion.](https://github.com/apache/spark/pull/24759#issuecomment-560211270) ### Why are the changes needed? To follow the SQL EXPLAIN command. ### Does this PR introduce any user-facing change? No, this is just for a new API in Dataset. ### How was this patch tested? Add tests in `ExplainSuite`. Closes #26829 from maropu/DatasetExplain. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 10 December 2019, 17:51:29 UTC
d9b3069 [SPARK-30125][SQL] Remove PostgreSQL dialect ### What changes were proposed in this pull request? Reprocess all PostgreSQL dialect related PRs, listing in order: - #25158: PostgreSQL integral division support [revert] - #25170: UT changes for the integral division support [revert] - #25458: Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type. [revert] - #25697: Combine below 2 feature tags into "spark.sql.dialect" [revert] - #26112: Date substraction support [keep the ANSI-compliant part] - #26444: Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" [revert] - #26463: Cast to boolean support for PostgreSQL dialect [revert] - #26584: Make the behavior of Postgre dialect independent of ansi mode config [keep the ANSI-compliant part] ### Why are the changes needed? As the discussion in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-PostgreSQL-dialect-td28417.html, we need to remove PostgreSQL dialect form code base for several reasons: 1. The current approach makes the codebase complicated and hard to maintain. 2. Fully migrating PostgreSQL workloads to Spark SQL is not our focus for now. ### Does this PR introduce any user-facing change? Yes, the config `spark.sql.dialect` will be removed. ### How was this patch tested? Existing UT. Closes #26763 from xuanyuanking/SPARK-30125. Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com> Co-authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 December 2019, 17:22:34 UTC
a9f1809 [SPARK-30206][SQL] Rename normalizeFilters in DataSourceStrategy to be generic ### What changes were proposed in this pull request? This PR renames `normalizeFilters` in `DataSourceStrategy` to be more generic as the logic is not specific to filters. ### Why are the changes needed? These changes are needed to support PR #26751. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26830 from aokolnychyi/rename-normalize-exprs. Authored-by: Anton Okolnychyi <aokolnychyi@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 10 December 2019, 15:49:22 UTC
1cac9b2 [SPARK-29967][ML][PYTHON] KMeans support instance weighting ### What changes were proposed in this pull request? add weight support in KMeans ### Why are the changes needed? KMeans should support weighting ### Does this PR introduce any user-facing change? Yes. ```KMeans.setWeightCol``` ### How was this patch tested? Unit Tests Closes #26739 from huaxingao/spark-29967. Authored-by: Huaxin Gao <huaxing@us.ibm.com> Signed-off-by: Sean Owen <srowen@gmail.com> 10 December 2019, 15:33:06 UTC
aa9da93 [SPARK-30151][SQL] Issue better error message when user-specified schema mismatched ### What changes were proposed in this pull request? Issue better error message when user-specified schema and not match relation schema ### Why are the changes needed? Inspired by https://github.com/apache/spark/pull/25248#issuecomment-559594305, user could get a weird error message when type mapping behavior change between Spark schema and datasource schema(e.g. JDBC). Instead of saying "SomeProvider does not allow user-specified schemas.", we'd better tell user what is really happening here to make user be more clearly about the error. ### Does this PR introduce any user-facing change? Yes, user will see error message changes. ### How was this patch tested? Updated existed tests. Closes #26781 from Ngone51/dev-mismatch-schema. Authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 December 2019, 12:56:21 UTC
be867e8 [SPARK-30196][BUILD] Bump lz4-java version to 1.7.0 ### What changes were proposed in this pull request? This pr intends to upgrade lz4-java from 1.6.0 to 1.7.0. ### Why are the changes needed? This release includes a performance bug (https://github.com/lz4/lz4-java/pull/143) fixed by JoshRosen and some improvements (e.g., LZ4 binary update). You can see the link below for the changes; https://github.com/lz4/lz4-java/blob/master/CHANGES.md#170 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26823 from maropu/LZ4_1_7_0. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 10 December 2019, 03:22:03 UTC
3d98c9f [SPARK-30179][SQL][TESTS] Improve test in SingleSessionSuite ### What changes were proposed in this pull request? improve the temporary functions test in SingleSessionSuite by verifying the result in a query ### Why are the changes needed? ### Does this PR introduce any user-facing change? ### How was this patch tested? Closes #26812 from leoluan2009/SPARK-30179. Authored-by: Luan <xuluan@ebay.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 10 December 2019, 01:57:32 UTC
36fa198 [SPARK-30158][SQL][CORE] Seq -> Array for sc.parallelize for 2.13 compatibility; remove WrappedArray ### What changes were proposed in this pull request? Use Seq instead of Array in sc.parallelize, with reference types. Remove usage of WrappedArray. ### Why are the changes needed? These both enable building on Scala 2.13. ### Does this PR introduce any user-facing change? None ### How was this patch tested? Existing tests Closes #26787 from srowen/SPARK-30158. Authored-by: Sean Owen <sean.owen@databricks.com> Signed-off-by: Sean Owen <srowen@gmail.com> 09 December 2019, 20:41:48 UTC
8a9cccf [SPARK-30146][ML][PYSPARK] Add setWeightCol to GBTs in PySpark ### What changes were proposed in this pull request? add ```setWeightCol``` and ```setMinWeightFractionPerNode``` in Python side of ```GBTClassifier``` and ```GBTRegressor``` ### Why are the changes needed? https://github.com/apache/spark/pull/25926 added ```setWeightCol``` and ```setMinWeightFractionPerNode``` in GBTs on scala side. This PR will add ```setWeightCol``` and ```setMinWeightFractionPerNode``` in GBTs on python side ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? doc test Closes #26774 from huaxingao/spark-30146. Authored-by: Huaxin Gao <huaxing@us.ibm.com> Signed-off-by: Sean Owen <srowen@gmail.com> 09 December 2019, 19:39:33 UTC
538b8d1 [SPARK-30159][SQL][FOLLOWUP] Fix lint-java via removing unnecessary imports ### What changes were proposed in this pull request? This patch fixes the Java code style violations in SPARK-30159 (#26788) which are caught by lint-java (Github Action caught it and I can reproduce it locally). Looks like Jenkins build may have different policy on checking Java style check or less accurate. ### Why are the changes needed? Java linter starts complaining. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? lint-java passed locally This closes #26819 Closes #26818 from HeartSaVioR/SPARK-30159-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 09 December 2019, 16:57:20 UTC
729f43f [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system ## What changes were proposed in this pull request? This PR proposes to add instrumentation of memory usage via the Spark Dropwizard/Codahale metrics system. Memory usage metrics are available via the Executor metrics, recently implemented as detailed in https://issues.apache.org/jira/browse/SPARK-23206. Additional notes: This takes advantage of the metrics poller introduced in #23767. ## Why are the changes needed? Executor metrics bring have many useful insights on memory usage, in particular on the usage of storage memory and executor memory. This is useful for troubleshooting. Having the information in the metrics systems allows to add those metrics to Spark performance dashboards and study memory usage as a function of time, as in the example graph https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG ## Does this PR introduce any user-facing change? Adds `ExecutorMetrics` source to publish executor metrics via the Dropwizard metrics system. Details of the available metrics in docs/monitoring.md Adds configuration parameter `spark.metrics.executormetrics.source.enabled` ## How was this patch tested? Tested on YARN cluster and with an existing setup for a Spark dashboard based on InfluxDB and Grafana. Closes #24132 from LucaCanali/memoryMetricsSource. Authored-by: Luca Canali <luca.canali@cern.ch> Signed-off-by: Imran Rashid <irashid@cloudera.com> 09 December 2019, 14:55:30 UTC
a717d21 [SPARK-30159][SQL][TESTS] Fix the method calls of `QueryTest.checkAnswer` ### What changes were proposed in this pull request? Before this PR, the method `checkAnswer` in Object `QueryTest` returns an optional string. It doesn't throw exceptions when errors happen. The actual exceptions are thrown in the trait `QueryTest`. However, there are some test suites(`StreamSuite`, `SessionStateSuite`, `BinaryFileFormatSuite`, etc.) that use the no-op method `QueryTest.checkAnswer` and expect it to fail test cases when the execution results don't match the expected answers. After this PR: 1. the method `checkAnswer` in Object `QueryTest` will fail tests on errors or unexpected results. 2. add a new method `getErrorMessageInCheckAnswer`, which is exactly the same as the previous version of `checkAnswer`. There are some test suites use this one to customize the test failure message. 3. for the test suites that extend the trait `QueryTest`, we should use the method `checkAnswer` directly, instead of calling the method from Object `QueryTest`. ### Why are the changes needed? We should fix these method calls to perform actual validations in test suites. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #26788 from gengliangwang/fixCheckAnswer. Authored-by: Gengliang Wang <gengliang.wang@databricks.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 09 December 2019, 13:19:08 UTC
c2f29d5 [SPARK-30138][SQL] Separate configuration key of max iterations for analyzer and optimizer ### What changes were proposed in this pull request? separate the configuration keys "spark.sql.optimizer.maxIterations" and "spark.sql.analyzer.maxIterations". ### Why are the changes needed? Currently, both Analyzer and Optimizer use conf "spark.sql.optimizer.maxIterations" to set the max iterations to run, which is a little confusing. It is clearer to add a new conf "spark.sql.analyzer.maxIterations" for analyzer max iterations. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Existing unit tests. Closes #26766 from fuwhu/SPARK-30138. Authored-by: fuwhu <bestwwg@163.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org> 09 December 2019, 10:43:32 UTC
dcea7a4 [SPARK-29883][SQL] Implement a helper method for aliasing bool_and() and bool_or() ### What changes were proposed in this pull request? This PR introduces a method `expressionWithAlias` in class `FunctionRegistry` which is used to register function's constructor. Currently, `expressionWithAlias` is used to register `BoolAnd` & `BoolOr`. ### Why are the changes needed? Error message is wrong when alias name is used for `BoolAnd` & `BoolOr`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tested manually. For query, `select every('true');` Output before this PR, > Error in query: cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 7; After this PR, > Error in query: cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7; Closes #26712 from amanomer/29883. Authored-by: Aman Omer <amanomer1996@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 09 December 2019, 05:23:16 UTC
a57bbf2 [SPARK-30164][TESTS][DOCS] Exclude Hive domain in Unidoc build explicitly ### What changes were proposed in this pull request? This PR proposes to exclude Unidoc checking in Hive domain. We don't publish this as a part of Spark documentation (see also https://github.com/apache/spark/blob/master/docs/_plugins/copy_api_dirs.rb#L30) and most of them are copy of Hive thrift server so that we can officially use Hive 2.3 release. It doesn't much make sense to check the documentation generation against another domain, and that we don't use in documentation publish. ### Why are the changes needed? To avoid unnecessary computation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? By Jenkins: ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-2.3 -Phive -Pmesos -Pkubernetes -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl -Pyarn test:package streaming-kinesis-asl-assembly/assembly ... ======================================================================== Building Unidoc API Documentation ======================================================================== [info] Building Spark unidoc using SBT with these arguments: -Phadoop-2.7 -Phive-2.3 -Phive -Pmesos -Pkubernetes -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl -Pyarn unidoc ... [info] Main Java API documentation successful. ... [info] Main Scala API documentation successful. ``` Closes #26800 from HyukjinKwon/do-not-merge. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 09 December 2019, 04:15:49 UTC
bca9de6 [SPARK-29922][SQL] SHOW FUNCTIONS should do multi-catalog resolution ### What changes were proposed in this pull request? Add ShowFunctionsStatement and make SHOW FUNCTIONS go through the same catalog/table resolution framework of v2 commands. We don’t have this methods in the catalog to implement an V2 command * catalog.listFunctions ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing `SHOW FUNCTIONS LIKE namespace.function` ### Does this PR introduce any user-facing change? Yes. When running SHOW FUNCTIONS LIKE namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26667 from planga82/feature/SPARK-29922_ShowFunctions_V2Catalog. Authored-by: Pablo Langa <soypab@gmail.com> Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com> 09 December 2019, 04:15:09 UTC
16f1b23 [SPARK-30163][INFRA][FOLLOWUP] Make `.m2` directory for cold start without cache ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/26793 and aims to initialize `~/.m2` directory. ### Why are the changes needed? In case of cache reset, `~/.m2` directory doesn't exist. It causes a failure. - `master` branch has a cache as of now. So, we missed this. - `branch-2.4` has no cache as of now, and we hit this failure. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR is tested against personal `branch-2.4`. - https://github.com/dongjoon-hyun/spark/pull/12 Closes #26794 from dongjoon-hyun/SPARK-30163-2. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 07 December 2019, 20:58:00 UTC
1068b8b [SPARK-30163][INFRA] Use Google Maven mirror in GitHub Action ### What changes were proposed in this pull request? This PR aims to use [Google Maven mirror](https://cloudplatform.googleblog.com/2015/11/faster-builds-for-Java-developers-with-Maven-Central-mirror.html) in `GitHub Action` jobs to improve the stability. ```xml <settings> <mirrors> <mirror> <id>google-maven-central</id> <name>GCS Maven Central mirror</name> <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url> <mirrorOf>central</mirrorOf> </mirror> </mirrors> </settings> ``` ### Why are the changes needed? Although we added Maven cache inside `GitHub Action`, the timeouts happen too frequently during access `artifact descriptor`. ``` [ERROR] Failed to execute goal on project spark-mllib_2.12: ... Failed to read artifact descriptor for ... ... Connection timed out (Read failed) -> [Help 1] ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR is irrelevant to Jenkins. This is tested on the personal repository first. `GitHub Action` of this PR should pass. - https://github.com/dongjoon-hyun/spark/pull/11 Closes #26793 from dongjoon-hyun/SPARK-30163. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 07 December 2019, 20:04:10 UTC
e88d740 [SPARK-30147][SQL] Trim the string when cast string type to booleans ### What changes were proposed in this pull request? Now, we trim the string when casting string value to those `canCast` types values, e.g. int, double, decimal, interval, date, timestamps, except for boolean. This behavior makes type cast and coercion inconsistency in Spark. Not fitting ANSI SQL standard either. ``` If TD is boolean, then Case: a) If SD is character string, then SV is replaced by TRIM ( BOTH ' ' FROM VE ) Case: i) If the rules for literal in Subclause 5.3, “literal”, can be applied to SV to determine a valid value of the data type TD, then let TV be that value. ii) Otherwise, an exception condition is raised: data exception — invalid character value for cast. b) If SD is boolean, then TV is SV ``` In this pull request, we trim all the whitespaces from both ends of the string before converting it to a bool value. This behavior is as same as others, but a bit different from sql standard, which trim only spaces. ### Why are the changes needed? Type cast/coercion consistency ### Does this PR introduce any user-facing change? yes, string with whitespaces in both ends will be trimmed before converted to booleans. e.g. `select cast('\t true' as boolean)` results `true` now, before this pr it's `null` ### How was this patch tested? add unit tests Closes #26776 from yaooqinn/SPARK-30147. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org> 07 December 2019, 06:03:51 UTC
afc4fa0 [SPARK-30156][BUILD] Upgrade Jersey from 2.29 to 2.29.1 ### What changes were proposed in this pull request? This PR aims to upgrade `Jersey` from 2.29 to 2.29.1. ### Why are the changes needed? This will bring several bug fixes and important dependency upgrades. - https://eclipse-ee4j.github.io/jersey.github.io/release-notes/2.29.1.html ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26785 from dongjoon-hyun/SPARK-30156. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 07 December 2019, 02:49:43 UTC
1e0037b [SPARK-30157][BUILD][TEST-HADOOP3.2][TEST-JAVA11] Upgrade Apache HttpCore from 4.4.10 to 4.4.12 ### What changes were proposed in this pull request? This PR aims to upgrade `Apache HttpCore` from 4.4.10 to 4.4.12. ### Why are the changes needed? `Apache HttpCore v4.4.11` is the first official release for JDK11. > This is a maintenance release that corrects a number of defects in non-blocking SSL session code that caused compatibility issues with TLSv1.3 protocol implementation shipped with Java 11. For the full release note, please see the following. - https://www.apache.org/dist/httpcomponents/httpcore/RELEASE_NOTES-4.4.x.txt ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26786 from dongjoon-hyun/SPARK-30157. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 07 December 2019, 01:59:10 UTC
51aa7a9 [SPARK-30148][SQL] Optimize writing plans if there is an analysis exception ### What changes were proposed in this pull request? Optimized QueryExecution.scala#writePlans(). ### Why are the changes needed? If any query fails in Analysis phase and gets AnalysisException, there is no need to execute further phases since those will return a same result i.e, AnalysisException. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26778 from amanomer/optExplain. Authored-by: Aman Omer <amanomer1996@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 07 December 2019, 01:58:02 UTC
a30ec19 [SPARK-30155][SQL] Rename parse() to parseString() to avoid conflict in Scala 2.13 ### What changes were proposed in this pull request? Rename internal method LegacyTypeStringParser.parse() to parseString(). ### Why are the changes needed? In Scala 2.13, the parse() definition clashes with supertype declarations. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26784 from srowen/SPARK-30155. Authored-by: Sean Owen <sean.owen@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 07 December 2019, 00:16:28 UTC
81996f9 [SPARK-30152][INFRA] Enable Hadoop-2.7/JDK11 build at GitHub Action ### What changes were proposed in this pull request? This PR enables JDK11 build with `hadoop-2.7` profile at `GitHub Action`. **BEFORE (6 jobs including one JDK11 job)** ![before](https://user-images.githubusercontent.com/9700541/70342731-7763f300-180a-11ea-859f-69038b88451f.png) **AFTER (7 jobs including two JDK11 jobs)** ![after](https://user-images.githubusercontent.com/9700541/70342658-54d1da00-180a-11ea-9fba-507fc087dc62.png) ### Why are the changes needed? SPARK-29957 makes JDK11 test work with `hadoop-2.7` profile. We need to protect it. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This is `GitHub Action` only PR. See the result of `GitHub Action` on this PR. Closes #26782 from dongjoon-hyun/SPARK-GHA-HADOOP-2.7. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 06 December 2019, 20:01:36 UTC
58be82a [SPARK-30098][SQL] Use default datasource as provider for CREATE TABLE syntax ### What changes were proposed in this pull request? In this PR, we propose to use the value of `spark.sql.source.default` as the provider for `CREATE TABLE` syntax instead of `hive` in Spark 3.0. And to help the migration, we introduce a legacy conf `spark.sql.legacy.respectHiveDefaultProvider.enabled` and set its default to `false`. ### Why are the changes needed? 1. Currently, `CREATE TABLE` syntax use hive provider to create table while `DataFrameWriter.saveAsTable` API using the value of `spark.sql.source.default` as a provider to create table. It would be better to make them consistent. 2. User may gets confused in some cases. For example: ``` CREATE TABLE t1 (c1 INT) USING PARQUET; CREATE TABLE t2 (c1 INT); ``` In these two DDLs, use may think that `t2` should also use parquet as default provider since Spark always advertise parquet as the default format. However, it's hive in this case. On the other hand, if we omit the USING clause in a CTAS statement, we do pick parquet by default if `spark.sql.hive.convertCATS=true`: ``` CREATE TABLE t3 USING PARQUET AS SELECT 1 AS VALUE; CREATE TABLE t4 AS SELECT 1 AS VALUE; ``` And these two cases together can be really confusing. 3. Now, Spark SQL is very independent and popular. We do not need to be fully consistent with Hive's behavior. ### Does this PR introduce any user-facing change? Yes, before this PR, using `CREATE TABLE` syntax will use hive provider. But now, it use the value of `spark.sql.source.default` as its provider. ### How was this patch tested? Added tests in `DDLParserSuite` and `HiveDDlSuite`. Closes #26736 from Ngone51/dev-create-table-using-parquet-by-default. Lead-authored-by: wuyi <yi.wu@databricks.com> Co-authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 06 December 2019, 18:15:25 UTC
c1a5f94 [SPARK-30112][SQL] Allow insert overwrite same table if using dynamic partition overwrite ### What changes were proposed in this pull request? This patch proposes to allow insert overwrite same table if using dynamic partition overwrite. ### Why are the changes needed? Currently, Insert overwrite cannot overwrite to same table even it is dynamic partition overwrite. But for dynamic partition overwrite, we do not delete partition directories ahead. We write to staging directories and move data to final partition directories. We should be able to insert overwrite to same table under dynamic partition overwrite. This enables users to read data from a table and insert overwrite to same table by using dynamic partition overwrite. Because this is not allowed for now, users need to write to other temporary location and move it back to the table. ### Does this PR introduce any user-facing change? Yes. Users can insert overwrite same table if using dynamic partition overwrite. ### How was this patch tested? Unit test. Closes #26752 from viirya/dynamic-overwrite-same-table. Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com> Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 06 December 2019, 17:22:16 UTC
c8ed71b [SPARK-30011][SQL] Inline 2.12 "AsIfIntegral" classes, not present in 2.13 ### What changes were proposed in this pull request? Classes like DoubleAsIfIntegral are not found in Scala 2.13, but used in the current build. This change 'inlines' the 2.12 implementation and makes it work with both 2.12 and 2.13. ### Why are the changes needed? To cross-compile with 2.13. ### Does this PR introduce any user-facing change? It should not as it copies in 2.12's existing behavior. ### How was this patch tested? Existing tests. Closes #26769 from srowen/SPARK-30011. Authored-by: Sean Owen <sean.owen@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 06 December 2019, 16:15:38 UTC
1595e46 [SPARK-30142][TEST-MAVEN][BUILD] Upgrade Maven to 3.6.3 ### What changes were proposed in this pull request? This PR aims to upgrade Maven from 3.6.2 to 3.6.3. ### Why are the changes needed? This will bring bug fixes like the following. - MNG-6759 Maven fails to use <repositories> section from dependency when resolving transitive dependencies in some cases - MNG-6760 ExclusionArtifactFilter result invalid when wildcard exclusion is followed by other exclusions The following is the full release note. - https://maven.apache.org/docs/3.6.3/release-notes.html ### Does this PR introduce any user-facing change? No. (This is a dev-environment change.) ### How was this patch tested? Pass the Jenkins with both SBT and Maven. Closes #26770 from dongjoon-hyun/SPARK-30142. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org> 06 December 2019, 14:41:59 UTC
187f3c1 [SPARK-28083][SQL] Support LIKE ... ESCAPE syntax ## What changes were proposed in this pull request? The syntax 'LIKE predicate: ESCAPE clause' is a ANSI SQL. For example: ``` select 'abcSpark_13sd' LIKE '%Spark\\_%'; //true select 'abcSpark_13sd' LIKE '%Spark/_%'; //false select 'abcSpark_13sd' LIKE '%Spark"_%'; //false select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/'; //true select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"'; //true select 'abcSpark%13sd' LIKE '%Spark\\%%'; //true select 'abcSpark%13sd' LIKE '%Spark/%%'; //false select 'abcSpark%13sd' LIKE '%Spark"%%'; //false select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/'; //true select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"'; //true select 'abcSpark\\13sd' LIKE '%Spark\\\\_%'; //true select 'abcSpark/13sd' LIKE '%Spark//_%'; //false select 'abcSpark"13sd' LIKE '%Spark""_%'; //false select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/'; //true select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"'; //true ``` But Spark SQL only supports 'LIKE predicate'. Note: If the input string or pattern string is null, then the result is null too. There are some mainstream database support the syntax. **PostgreSQL:** https://www.postgresql.org/docs/11/functions-matching.html **Vertica:** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Predicates/LIKE-predicate.htm?zoom_highlight=like%20escape **MySQL:** https://dev.mysql.com/doc/refman/5.6/en/string-comparison-functions.html **Oracle:** https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/JDBC-reference-information.html#GUID-5D371A5B-D7F6-42EB-8C0D-D317F3C53708 https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Pattern-matching-Conditions.html#GUID-0779657B-06A8-441F-90C5-044B47862A0A ## How was this patch tested? Exists UT and new UT. This PR merged to my production environment and runs above sql: ``` spark-sql> select 'abcSpark_13sd' LIKE '%Spark\\_%'; true Time taken: 0.119 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%'; false Time taken: 0.103 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%'; false Time taken: 0.096 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/'; true Time taken: 0.096 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"'; true Time taken: 0.092 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark\\%%'; true Time taken: 0.109 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%'; false Time taken: 0.1 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%'; false Time taken: 0.081 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/'; true Time taken: 0.095 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"'; true Time taken: 0.113 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark\\13sd' LIKE '%Spark\\\\_%'; true Time taken: 0.078 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%'; false Time taken: 0.067 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%'; false Time taken: 0.084 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/'; true Time taken: 0.091 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"'; true Time taken: 0.091 seconds, Fetched 1 row(s) ``` I create a table and its schema is: ``` spark-sql> desc formatted gja_test; key string NULL value string NULL other string NULL # Detailed Table Information Database test Table gja_test Owner test Created Time Wed Apr 10 11:06:15 CST 2019 Last Access Thu Jan 01 08:00:00 CST 1970 Created By Spark 2.4.1-SNAPSHOT Type MANAGED Provider hive Table Properties [transient_lastDdlTime=1563443838] Statistics 26 bytes Location hdfs://namenode.xxx:9000/home/test/hive/warehouse/test.db/gja_test Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.TextInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat Storage Properties [field.delim= , serialization.format= ] Partition Provider Catalog Time taken: 0.642 seconds, Fetched 21 row(s) ``` Table `gja_test` exists three rows of data. ``` spark-sql> select * from gja_test; a A ao b B bo "__ """__ " Time taken: 0.665 seconds, Fetched 3 row(s) ``` At finally, I test this function: ``` spark-sql> select * from gja_test where key like value escape '"'; "__ """__ " Time taken: 0.687 seconds, Fetched 1 row(s) ``` Closes #25001 from beliefer/ansi-sql-like. Lead-authored-by: gengjiaan <gengjiaan@360.cn> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com> 06 December 2019, 08:07:38 UTC
b86d4bb [SPARK-30001][SQL] ResolveRelations should handle both V1 and V2 tables ### What changes were proposed in this pull request? This PR makes `Analyzer.ResolveRelations` responsible for looking up both v1 and v2 tables from the session catalog and create an appropriate relation. ### Why are the changes needed? Currently there are two issues: 1. As described in [SPARK-29966](https://issues.apache.org/jira/browse/SPARK-29966), the logic for resolving relation can load a table twice, which is a perf regression (e.g., Hive metastore can be accessed twice). 2. As described in [SPARK-30001](https://issues.apache.org/jira/browse/SPARK-30001), if a catalog name is specified for v1 tables, the query fails: ``` scala> sql("create table t using csv as select 1 as i") res2: org.apache.spark.sql.DataFrame = [] scala> sql("select * from t").show +---+ | i| +---+ | 1| +---+ scala> sql("select * from spark_catalog.t").show org.apache.spark.sql.AnalysisException: Table or view not found: spark_catalog.t; line 1 pos 14; 'Project [*] +- 'UnresolvedRelation [spark_catalog, t] ``` ### Does this PR introduce any user-facing change? Yes. Now the catalog name is resolved correctly: ``` scala> sql("create table t using csv as select 1 as i") res0: org.apache.spark.sql.DataFrame = [] scala> sql("select * from t").show +---+ | i| +---+ | 1| +---+ scala> sql("select * from spark_catalog.t").show +---+ | i| +---+ | 1| +---+ ``` ### How was this patch tested? Added new tests. Closes #26684 from imback82/resolve_relation. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 06 December 2019, 07:45:13 UTC
a5ccbce [SPARK-30067][CORE] Fix fragment offset comparison in getBlockHosts ### What changes were proposed in this pull request? A bug fixed about the code in getBlockHosts() function. In the case "The fragment ends at a position within this block", the end of fragment should be before the end of block,where the "end of block" means `b.getOffset + b.getLength`,not `b.getLength`. ### Why are the changes needed? When comparing the fragment end and the block end,we should use fragment's `offset + length`,and then compare to the block's `b.getOffset + b.getLength`, not the block's length. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? No test. Closes #26650 from mdianjun/fix-getBlockHosts. Authored-by: madianjun <madianjun@jd.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 06 December 2019, 07:39:49 UTC
da27f91 [SPARK-29957][TEST] Reset MiniKDC's default enctypes to fit jdk8/jdk11 ### What changes were proposed in this pull request? Hadoop jira: https://issues.apache.org/jira/browse/HADOOP-12911 In this jira, the author said to replace origin Apache Directory project which is not maintained (but not said it won't work well in jdk11) to Apache Kerby which is java binding(fit java version). And in Flink: https://github.com/apache/flink/pull/9622 Author show the reason why hadoop-2.7.2's `MminiKdc` failed with jdk11. Because new encryption types of `es128-cts-hmac-sha256-128` and `aes256-cts-hmac-sha384-192` (for Kerberos 5) enabled by default were added in Java 11. Spark with `hadoop-2.7's MiniKdc`does not support these encryption types and does not work well when these encryption types are enabled, which results in the authentication failure. And when I test hadoop-2.7.2's minikdc in local, the kerberos 's debug error message is read message stream failed, message can't match. ### Why are the changes needed? Support jdk11 with hadoop-2.7 ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Existed UT Closes #26594 from AngersZhuuuu/minikdc-3.2.0. Lead-authored-by: angerszhu <angers.zhu@gmail.com> Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 06 December 2019, 07:12:45 UTC
25431d7 [SPARK-29953][SS] Don't clean up source files for FileStreamSource if the files belong to the output of FileStreamSink ### What changes were proposed in this pull request? This patch prevents the cleanup operation in FileStreamSource if the source files belong to the FileStreamSink. This is needed because the output of FileStreamSink can be read with multiple Spark queries and queries will read the files based on the metadata log, which won't reflect the cleanup. To simplify the logic, the patch only takes care of the case of when the source path without glob pattern refers to the output directory of FileStreamSink, via checking FileStreamSource to see whether it leverages metadata directory or not to list the source files. ### Why are the changes needed? Without this patch, if end users turn on cleanup option with the path which is the output of FileStreamSink, there may be out of sync between metadata and available files which may break other queries reading the path. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added UT. Closes #26590 from HeartSaVioR/SPARK-29953. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: Shixiong Zhu <zsxwing@gmail.com> 06 December 2019, 05:46:28 UTC
755d889 [SPARK-24666][ML] Fix infinity vectors produced by Word2Vec when numIterations are large ### What changes were proposed in this pull request? This patch adds normalization to word vectors when fitting dataset in Word2Vec. ### Why are the changes needed? Running Word2Vec on some datasets, when numIterations is large, can produce infinity word vectors. ### Does this PR introduce any user-facing change? Yes. After this patch, Word2Vec won't produce infinity word vectors. ### How was this patch tested? Manually. This issue is not always reproducible on any dataset. The dataset known to reproduce it is too large (925M) to upload. ```scala case class Sentences(name: String, words: Array[String]) val dataset = spark.read .option("header", "true").option("sep", "\t") .option("quote", "").option("nullValue", "\\N") .csv("/tmp/title.akas.tsv") .filter("region = 'US' or language = 'en'") .select("title") .as[String] .map(s => Sentences(s, s.split(' '))) .persist() println("Training model...") val word2Vec = new Word2Vec() .setInputCol("words") .setOutputCol("vector") .setVectorSize(64) .setWindowSize(4) .setNumPartitions(50) .setMinCount(5) .setMaxIter(30) val model = word2Vec.fit(dataset) model.getVectors.show() ``` Before: ``` Training model... +-------------+--------------------+ | word| vector| +-------------+--------------------+ | Unspoken|[-Infinity,-Infin...| | Talent|[-Infinity,Infini...| | Hourglass|[2.02805806500023...| |Nickelodeon's|[-4.2918617120906...| | Priests|[-1.3570403355926...| | Religion:|[-6.7049072282803...| | Bu|[5.05591774315586...| | Totoro:|[-1.0539840178632...| | Trouble,|[-3.5363592836003...| | Hatter|[4.90413981352826...| | '79|[7.50436471285412...| | Vile|[-2.9147142985312...| | 9/11|[-Infinity,Infini...| | Santino|[1.30005911270850...| | Motives|[-1.2538958306253...| | '13|[-4.5040152427657...| | Fierce|[Infinity,Infinit...| | Stover|[-2.6326895394029...| | 'It|[1.66574533864436...| | Butts|[Infinity,Infinit...| +-------------+--------------------+ only showing top 20 rows ``` After: ``` Training model... +-------------+--------------------+ | word| vector| +-------------+--------------------+ | Unspoken|[-0.0454501919448...| | Talent|[-0.2657704949378...| | Hourglass|[-0.1399687677621...| |Nickelodeon's|[-0.1767119318246...| | Priests|[-0.0047509293071...| | Religion:|[-0.0411605164408...| | Bu|[0.11837736517190...| | Totoro:|[0.05258282646536...| | Trouble,|[0.09482011198997...| | Hatter|[0.06040831282734...| | '79|[0.04783720895648...| | Vile|[-0.0017210749210...| | 9/11|[-0.0713915303349...| | Santino|[-0.0412711687386...| | Motives|[-0.0492418706417...| | '13|[-0.0073119504377...| | Fierce|[-0.0565455369651...| | Stover|[0.06938160210847...| | 'It|[0.01117012929171...| | Butts|[0.05374567210674...| +-------------+--------------------+ only showing top 20 rows ``` Closes #26722 from viirya/SPARK-24666-2. Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com> Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com> 06 December 2019, 00:32:33 UTC
7782b61 [SPARK-29392][CORE][SQL][FOLLOWUP] Avoid deprecated (in 2.13) Symbol syntax 'foo in favor of simpler expression, where it generated deprecation warnings TL;DR - this is more of the same change in https://github.com/apache/spark/pull/26748 I told you it'd be iterative! Closes #26765 from srowen/SPARK-29392.3. Authored-by: Sean Owen <sean.owen@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> 05 December 2019, 21:48:29 UTC
5892bbf [SPARK-30124][MLLIB] unnecessary persist in PythonMLLibAPI.scala ### What changes were proposed in this pull request? Removed unnecessary persist. ### Why are the changes needed? Persist in `PythonMLLibAPI.scala` is unnecessary because later in `run()` of `gmmAlg` is caching the data. https://github.com/apache/spark/blob/710ddab39e20f49e917311c3e27d142b5a2bcc71/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala#L167-L171 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26758 from amanomer/improperPersist. Authored-by: Aman Omer <amanomer1996@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com> 05 December 2019, 17:54:45 UTC
35bab33 [SPARK-30121][BUILD] Fix memory usage in sbt build script ### What changes were proposed in this pull request? 1. the default memory setting is missing in usage instructions ``` build/sbt -h ``` before ``` -mem <integer> set memory options (default: , which is -Xms2048m -Xmx2048m -XX:ReservedCodeCacheSize=256m) ``` after ``` -mem <integer> set memory options (default: 2048, which is -Xms2048m -Xmx2048m -XX:ReservedCodeCacheSize=256m) ``` 2. the Perm space is not needed anymore, since java7 is removed. the changes in this pr are based on the main sbt script of the newest stable version 1.3.4. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? no ### How was this patch tested? manually Closes #26757 from yaooqinn/SPARK-30121. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com> 05 December 2019, 17:50:55 UTC
b9cae37 [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres # What changes were proposed in this pull request? Add an analyzer rule to convert unresolved `Add`, `Subtract`, etc. to `TimeAdd`, `DateAdd`, etc. according to the following policy: ```scala /** * For [[Add]]: * 1. if both side are interval, stays the same; * 2. else if one side is interval, turns it to [[TimeAdd]]; * 3. else if one side is date, turns it to [[DateAdd]] ; * 4. else stays the same. * * For [[Subtract]]: * 1. if both side are interval, stays the same; * 2. else if the right side is an interval, turns it to [[TimeSub]]; * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]]; * 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]]; * 5. else if the left side is date, turns it to [[DateSub]]; * 6. else turns it to stays the same. * * For [[Multiply]]: * 1. If one side is interval, turns it to [[MultiplyInterval]]; * 2. otherwise, stays the same. * * For [[Divide]]: * 1. If the left side is interval, turns it to [[DivideInterval]]; * 2. otherwise, stays the same. */ ``` Besides, we change datetime functions from implicit cast types to strict ones, all available type coercions happen in `DateTimeOperations` coercion rule. ### Why are the changes needed? Feature Parity between PostgreSQL and Spark, and make the null semantic consistent with Spark. ### Does this PR introduce any user-facing change? 1. date_add/date_sub functions only accept int/tinynit/smallint as the second arg, double/string etc, are forbidden like hive, which produce weird results. ### How was this patch tested? add ut Closes #26412 from yaooqinn/SPARK-29774. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 05 December 2019, 14:03:44 UTC
332e252 [SPARK-29425][SQL] The ownership of a database should be respected ### What changes were proposed in this pull request? Keep the owner of a database when executing alter database commands ### Why are the changes needed? Spark will inadvertently delete the owner of a database for executing databases ddls ### Does this PR introduce any user-facing change? NO ### How was this patch tested? add and modify uts Closes #26080 from yaooqinn/SPARK-29425. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 05 December 2019, 08:14:27 UTC
0ab922c [SPARK-29860][SQL] Fix dataType mismatch issue for InSubquery ### What changes were proposed in this pull request? There is an issue for InSubquery expression. For example, there are two tables `ta` and `tb` created by the below statements. ``` sql("create table ta(id Decimal(18,0)) using parquet") sql("create table tb(id Decimal(19,0)) using parquet") ``` This statement below would thrown dataType mismatch exception. ``` sql("select * from ta where id in (select id from tb)").show() ``` However, this similar statement could execute successfully. ``` sql("select * from ta where id in ((select id from tb))").show() ``` The root cause is that, for `InSubquery` expression, it does not find a common type for two decimalType like `In` expression. Besides that, for `InSubquery` expression, it also does not find a common type for DecimalType and double/float/bigInt. In this PR, I fix this issue by finding widerType for `InSubquery` expression when DecimalType is involved. ### Why are the changes needed? Some InSubquery would throw dataType mismatch exception. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #26485 from turboFei/SPARK-29860-in-subquery. Authored-by: turbofei <fwang12@ebay.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 05 December 2019, 08:00:16 UTC
0bd8b99 [SPARK-30093][SQL] Improve error message for creating view ### What changes were proposed in this pull request? Improved error message while creating views. ### Why are the changes needed? Error message should suggest user to use TEMPORARY keyword while creating permanent view referred by temporary view. https://github.com/apache/spark/pull/26317#discussion_r352377363 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Updated test case. Closes #26731 from amanomer/imp_err_msg. Authored-by: Aman Omer <amanomer1996@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> 05 December 2019, 07:28:07 UTC
back to top