sort by:
Revision Author Date Message Commit Date
f2408bd [SPARK-20843][CORE] Add a config to set driver terminate timeout ## What changes were proposed in this pull request? Add a `worker` configuration to set how long to wait before forcibly killing driver. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #18126 from zsxwing/SPARK-20843. (cherry picked from commit 6c1dbd6fc8d49acf7c1c902d2ebf89ed5e788a4e) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 27 May 2017, 05:25:44 UTC
39f7665 [SPARK-19659][CORE][FOLLOW-UP] Fetch big blocks to disk when shuffle-read ## What changes were proposed in this pull request? This PR includes some minor improvement for the comments and tests in https://github.com/apache/spark/pull/16989 ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #18117 from cloud-fan/follow. (cherry picked from commit 1d62f8aca82601506c44b6fd852f4faf3602d7e2) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 27 May 2017, 02:58:08 UTC
fc799d7 [SPARK-10643][CORE] Make spark-submit download remote files to local in client mode ## What changes were proposed in this pull request? This PR makes spark-submit script download remote files to local file system for local/standalone client mode. ## How was this patch tested? - Unit tests - Manual tests by adding s3a jar and testing against file on s3. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Yu Peng <loneknightpy@gmail.com> Closes #18078 from loneknightpy/download-jar-in-spark-submit. (cherry picked from commit 4af37812915763ac3bfd91a600a7f00a4b84d29a) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 26 May 2017, 23:29:25 UTC
30922de [SPARK-20694][DOCS][SQL] Document DataFrameWriter partitionBy, bucketBy and sortBy in SQL guide ## What changes were proposed in this pull request? - Add Scala, Python and Java examples for `partitionBy`, `sortBy` and `bucketBy`. - Add _Bucketing, Sorting and Partitioning_ section to SQL Programming Guide - Remove bucketing from Unsupported Hive Functionalities. ## How was this patch tested? Manual tests, docs build. Author: zero323 <zero323@users.noreply.github.com> Closes #17938 from zero323/DOCS-BUCKETING-AND-PARTITIONING. (cherry picked from commit ae33abf71b353c638487948b775e966c7127cd46) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 26 May 2017, 22:01:23 UTC
2b59ed4 [SPARK-20844] Remove experimental from Structured Streaming APIs Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate. I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3. Author: Michael Armbrust <michael@databricks.com> Closes #18065 from marmbrus/streamingGA. 26 May 2017, 20:34:33 UTC
92837ae [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit ## What changes were proposed in this pull request? When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails. This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught. ## How was this patch tested? Add a test suite into `DataFrameSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17087 from kiszk/SPARK-19372. 26 May 2017, 18:15:21 UTC
f99456b [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities ## What changes were proposed in this pull request? Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest. ## How was this patch tested? Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages. Author: NICHOLAS T. MARION <nmarion@us.ibm.com> Closes #17686 from n-marion/xss-fix. (cherry picked from commit b512233a457092b0e2a39d0b42cb021abc69d375) Signed-off-by: Sean Owen <sowen@cloudera.com> 26 May 2017, 18:11:18 UTC
fafe283 [SPARK-20868][CORE] UnsafeShuffleWriter should verify the position after FileChannel.transferTo ## What changes were proposed in this pull request? Long time ago we fixed a [bug](https://issues.apache.org/jira/browse/SPARK-3948) in shuffle writer about `FileChannel.transferTo`. We were not very confident about that fix, so we added a position check after the writing, try to discover the bug earlier. However this checking is missing in the new `UnsafeShuffleWriter`, this PR adds it. https://issues.apache.org/jira/browse/SPARK-18105 maybe related to that `FileChannel.transferTo` bug, hopefully we can find out the root cause after adding this position check. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #18091 from cloud-fan/shuffle. (cherry picked from commit d9ad78908f6189719cec69d34557f1a750d2e6af) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 26 May 2017, 07:01:42 UTC
289dd17 [SPARK-20888][SQL][DOCS] Document change of default setting of spark.sql.hive.caseSensitiveInferenceMode (Link to Jira: https://issues.apache.org/jira/browse/SPARK-20888) ## What changes were proposed in this pull request? Document change of default setting of spark.sql.hive.caseSensitiveInferenceMode configuration key from NEVER_INFO to INFER_AND_SAVE in the Spark SQL 2.1 to 2.2 migration notes. Author: Michael Allman <michael@videoamp.com> Closes #18112 from mallman/spark-20888-document_infer_and_save. (cherry picked from commit c1e7989c4ffd83c51f5c97998b4ff6fe8dd83cf4) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 26 May 2017, 01:26:16 UTC
7a21de9 [SPARK-20874][EXAMPLES] Add Structured Streaming Kafka Source to examples project ## What changes were proposed in this pull request? Add Structured Streaming Kafka Source to the `examples` project so that people can run `bin/run-example StructuredKafkaWordCount ...`. ## How was this patch tested? manually tested it. Author: Shixiong Zhu <shixiong@databricks.com> Closes #18101 from zsxwing/add-missing-example-dep. (cherry picked from commit 98c3852986a2cb5f2d249d6c8ef602be283bd90e) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 25 May 2017, 17:49:23 UTC
5ae1c65 [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path check for sc.addJar on Windows ## What changes were proposed in this pull request? This PR proposes two things: - A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar on Windows as well). ``` org.apache.spark.SparkContextSuite: - add jar with invalid path *** FAILED *** (32 milliseconds) 2 was not equal to 1 (SparkContextSuite.scala:309) ... ``` - Fix path vs URI related test failures on Windows. ``` org.apache.spark.storage.LocalDirsSuite: - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds) new java.io.File("/NONEXISTENT_PATH").exists() was true (LocalDirsSuite.scala:50) ... - Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved *** FAILED *** (15 milliseconds) Expected exception java.io.IOException to be thrown, but no exception was thrown. (LocalDirsSuite.scala:64) ... ``` ``` org.apache.spark.sql.hive.HiveSchemaInferenceSuite: - orc: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254 ... - parquet: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939 ... - orc: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (141 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c ... - parquet: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (125 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc ... - orc: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (156 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a ... - parquet: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (547 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee ... ``` ``` org.apache.spark.sql.execution.command.DDLSuite: - create temporary view using *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-3881d9ca-561b-488d-90b9-97587472b853 mp; ... - insert data to a data source table which has a non-existing location should succeed *** FAILED *** (109 milliseconds) file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 did not equal file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 (DDLSuite.scala:1869) ... - insert into a data source table with a non-existing partition location should succeed *** FAILED *** (94 milliseconds) file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d did not equal file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d (DDLSuite.scala:1910) ... - read data from a data source table which has a non-existing location should succeed *** FAILED *** (93 milliseconds) file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 did not equal file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 (DDLSuite.scala:1937) ... - read data from a data source table with non-existing partition location should succeed *** FAILED *** (110 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - create datasource table with a non-existing location *** FAILED *** (94 milliseconds) file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 did not equal file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 (DDLSuite.scala:1982) ... - CTAS for external data source table with a non-existing location *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - CTAS for external data source table with a existed location *** FAILED *** (15 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - data source table:partition column name containing a b *** FAILED *** (125 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - data source table:partition column name containing a:b *** FAILED *** (143 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - data source table:partition column name containing a%b *** FAILED *** (109 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - data source table:partition column name containing a,b *** FAILED *** (109 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - location uri contains a b for datasource table *** FAILED *** (94 milliseconds) file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b did not equal file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b (DDLSuite.scala:2084) ... - location uri contains a:b for datasource table *** FAILED *** (78 milliseconds) file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b did not equal file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b (DDLSuite.scala:2084) ... - location uri contains a%b for datasource table *** FAILED *** (78 milliseconds) file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b did not equal file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b (DDLSuite.scala:2084) ... - location uri contains a b for database *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - location uri contains a:b for database *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - location uri contains a%b for database *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... ``` ``` org.apache.spark.sql.hive.execution.HiveDDLSuite: - create hive table with a non-existing location *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - CTAS for external hive table with a non-existing location *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - CTAS for external hive table with a existed location *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - partition column name of parquet table containing a b *** FAILED *** (156 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - partition column name of parquet table containing a:b *** FAILED *** (94 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - partition column name of parquet table containing a%b *** FAILED *** (125 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - partition column name of parquet table containing a,b *** FAILED *** (110 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... - partition column name of hive table containing a b *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - partition column name of hive table containing a:b *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - partition column name of hive table containing a%b *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - partition column name of hive table containing a,b *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - hive table: location uri contains a b *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - hive table: location uri contains a:b *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... - hive table: location uri contains a%b *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ... ``` ``` org.apache.spark.sql.sources.PathOptionSuite: - path option also exist for write path *** FAILED *** (94 milliseconds) file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc did not equal file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc (PathOptionSuite.scala:98) ... ``` ``` org.apache.spark.sql.CachedTableSuite: - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table *** FAILED *** (110 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ... ``` ``` org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite: - treeString is redacted *** FAILED *** (250 milliseconds) "file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" did not contain "C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" (DataSourceScanExecRedactionSuite.scala:46) ... ``` ## How was this patch tested? Tested via AppVeyor for each and checked it passed once each. These should be retested via AppVeyor in this PR. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17987 from HyukjinKwon/windows-20170515. (cherry picked from commit e9f983df275c138626af35fd263a7abedf69297f) Signed-off-by: Sean Owen <sowen@cloudera.com> 25 May 2017, 16:10:38 UTC
022a495 [SPARK-20741][SPARK SUBMIT] Added cleanup of JARs archive generated by SparkSubmit ## What changes were proposed in this pull request? Deleted generated JARs archive after distribution to HDFS ## How was this patch tested? Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Lior Regev <lioregev@gmail.com> Closes #17986 from liorregev/master. (cherry picked from commit 7306d556903c832984c7f34f1e8fe738a4b2343c) Signed-off-by: Sean Owen <sowen@cloudera.com> 25 May 2017, 16:08:41 UTC
e01f1f2 [SPARK-20768][PYSPARK][ML] Expose numPartitions (expert) param of PySpark FPGrowth. ## What changes were proposed in this pull request? Expose numPartitions (expert) param of PySpark FPGrowth. ## How was this patch tested? + [x] Pass all unit tests. Author: Yan Facai (颜发才) <facai.yan@gmail.com> Closes #18058 from facaiy/ENH/pyspark_fpg_add_num_partition. (cherry picked from commit 139da116f130ed21481d3e9bdee5df4b8d7760ac) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 25 May 2017, 13:40:52 UTC
9cbf39f [SPARK-19281][FOLLOWUP][ML] Minor fix for PySpark FPGrowth. ## What changes were proposed in this pull request? Follow-up for #17218, some minor fix for PySpark ```FPGrowth```. ## How was this patch tested? Existing UT. Author: Yanbo Liang <ybliang8@gmail.com> Closes #18089 from yanboliang/spark-19281. (cherry picked from commit 913a6bfe4b0eb6b80a03b858ab4b2767194103de) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 25 May 2017, 12:15:38 UTC
8896c4e [SPARK-19659] Fetch big blocks to disk when shuffle-read. ## What changes were proposed in this pull request? Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse. Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used). The hope is strong especially when migrating data engine to Spark from another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one is very time consuming. It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than kill the job because of OOM. In this pr, I propose to fetch big blocks to disk(which is also mentioned in SPARK-3019): 1. Track average size and also the outliers(which are larger than 2*avgSize) in MapStatus; 2. Request memory from `MemoryManager` before fetch blocks and release the memory to `MemoryManager` when `ManagedBuffer` is released. 3. Fetch remote blocks to disk when failing acquiring memory from `MemoryManager`, otherwise fetch to memory. This is an improvement for memory control when shuffle blocks and help to avoid OOM in scenarios like below: 1. Single huge block; 2. Sizes of many blocks are underestimated in `MapStatus` and the actual footprint of blocks is much larger than the estimated. ## How was this patch tested? Added unit test in `MapStatusSuite` and `ShuffleBlockFetcherIteratorSuite`. Author: jinxing <jinxing6042@126.com> Closes #16989 from jinxing64/SPARK-19659. (cherry picked from commit 3f94e64aa8fd806ae1fa0156d846ce96afacddd3) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 25 May 2017, 08:11:51 UTC
b52a06d [SPARK-20250][CORE] Improper OOM error when a task been killed while spilling data ## What changes were proposed in this pull request? Currently, when a task is calling spill() but it receives a killing request from driver (e.g., speculative task), the `TaskMemoryManager` will throw an `OOM` exception. And we don't catch `Fatal` exception when a error caused by `Thread.interrupt`. So for `ClosedByInterruptException`, we should throw `RuntimeException` instead of `OutOfMemoryError`. https://issues.apache.org/jira/browse/SPARK-20250?jql=project%20%3D%20SPARK ## How was this patch tested? Existing unit tests. Author: Xianyang Liu <xianyang.liu@intel.com> Closes #18090 from ConeyLiu/SPARK-20250. (cherry picked from commit 731462a04f8e33ac507ad19b4270c783a012a33e) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 25 May 2017, 07:48:16 UTC
e0aa239 [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files ## What changes were proposed in this pull request? This is a follow-up to #18073. Taking a safer approach to shutdown the pool to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to set a better thread name. ## How was this patch tested? Manually test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18100 from viirya/SPARK-20848-followup. (cherry picked from commit 6b68d61cf31748a088778dfdd66491b2f89a3c7b) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 25 May 2017, 01:55:59 UTC
3f82d65 [SPARK-20403][SQL] Modify the instructions of some functions ## What changes were proposed in this pull request? 1. add instructions of 'cast' function When using 'show functions' and 'desc function cast' command in spark-sql 2. Modify the instructions of functions,such as boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string ## How was this patch tested? Before modification: spark-sql>desc function boolean; Function: boolean Class: org.apache.spark.sql.catalyst.expressions.Cast Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`. After modification: spark-sql> desc function boolean; Function: boolean Class: org.apache.spark.sql.catalyst.expressions.Cast Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. spark-sql> desc function cast Function: cast Class: org.apache.spark.sql.catalyst.expressions.Cast Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`. Author: liuxian <liu.xian3@zte.com.cn> Closes #17698 from 10110346/wip_lx_0418. (cherry picked from commit 197f9018a4641c8fc0725905ebfb535b61bed791) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 25 May 2017, 00:32:18 UTC
ae65d30 [SPARK-16202][SQL][DOC] Follow-up to Correct The Description of CreatableRelationProvider's createRelation ## What changes were proposed in this pull request? Follow-up to SPARK-16202: 1. Remove the duplication of the meaning of `SaveMode` (as one was in fact missing that had proven that the duplication may be incomplete in the future again) 2. Use standard scaladoc tags /cc gatorsmile rxin yhuai (as they were involved previously) ## How was this patch tested? local build Author: Jacek Laskowski <jacek@japila.pl> Closes #18026 from jaceklaskowski/CreatableRelationProvider-SPARK-16202. (cherry picked from commit 5f8ff2fc9a859ceeaa8f1d03060fdbb30951e706) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 25 May 2017, 00:24:33 UTC
2405afc [SPARK-20872][SQL] ShuffleExchange.nodeName should handle null coordinator ## What changes were proposed in this pull request? A one-liner change in `ShuffleExchange.nodeName` to cover the case when `coordinator` is `null`, so that the match expression is exhaustive. Please refer to [SPARK-20872](https://issues.apache.org/jira/browse/SPARK-20872) for a description of the symptoms. TL;DR is that inspecting a `ShuffleExchange` (directly or transitively) on the Executor side can hit a case where the `coordinator` field of a `ShuffleExchange` is null, and thus will trigger a `MatchError` in `ShuffleExchange.nodeName()`'s inexhaustive match expression. Also changed two other match conditions in `ShuffleExchange` on the `coordinator` field to be consistent. ## How was this patch tested? Manually tested this change with a case where the `coordinator` is null to make sure `ShuffleExchange.nodeName` doesn't throw a `MatchError` any more. Author: Kris Mok <kris.mok@databricks.com> Closes #18095 from rednaxelafx/shuffleexchange-nodename. (cherry picked from commit c0b3e45e3b46a5235b748cb85ad200c9ec1bb426) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 25 May 2017, 00:19:46 UTC
b7a2a16 [SPARK-20867][SQL] Move hints from Statistics into HintInfo class ## What changes were proposed in this pull request? This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future. ## How was this patch tested? Updated test cases to reflect the change. Author: Reynold Xin <rxin@databricks.com> Closes #18087 from rxin/SPARK-20867. (cherry picked from commit a64746677bf09ef67e3fd538355a6ee9b5ce8cf4) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 24 May 2017, 20:57:28 UTC
c59ad42 [SPARK-20848][SQL] Shutdown the pool after reading parquet files ## What changes were proposed in this pull request? From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is created. One of the threads in the pool is kept in the WAITING state, and never stopped, which leads to unbounded growth in number of threads. We should shutdown the pool after reading parquet files. ## How was this patch tested? Added a test to ParquetFileFormatSuite. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18073 from viirya/SPARK-20848. (cherry picked from commit f72ad303f05a6d99513ea3b121375726b177199c) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 24 May 2017, 16:36:01 UTC
83aeac9 [SPARK-20862][MLLIB][PYTHON] Avoid passing float to ndarray.reshape in LogisticRegressionModel ## What changes were proposed in this pull request? Fixed TypeError with python3 and numpy 1.12.1. Numpy's `reshape` no longer takes floats as arguments as of 1.12. Also, python3 uses float division for `/`, we should be using `//` to ensure that `_dataWithBiasSize` doesn't get set to a float. ## How was this patch tested? Existing tests run using python3 and numpy 1.12. Author: Bago Amirbekian <bago@databricks.com> Closes #18081 from MrBago/BF-py3floatbug. (cherry picked from commit bc66a77bbe2120cc21bd8da25194efca4cde13c3) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 24 May 2017, 14:56:28 UTC
1d10724 [SPARK-20631][FOLLOW-UP] Fix incorrect tests. ## What changes were proposed in this pull request? - Fix incorrect tests for `_check_thresholds`. - Move test to `ParamTests`. ## How was this patch tested? Unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #18085 from zero323/SPARK-20631-FOLLOW-UP. (cherry picked from commit 1816eb3bef930407dc9e083de08f5105725c55d1) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 24 May 2017, 11:58:40 UTC
e936a96 [SPARK-20764][ML][PYSPARK][FOLLOWUP] Fix visibility discrepancy with numInstances and degreesOfFreedom in LR and GLR - Python version ## What changes were proposed in this pull request? Add test cases for PR-18062 ## How was this patch tested? The existing UT Author: Peng <peng.meng@intel.com> Closes #18068 from mpjlu/moreTest. (cherry picked from commit 9afcf127d31b5477a539dde6e5f01861532a1c4c) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 24 May 2017, 11:54:58 UTC
ee9d597 [SPARK-18406][CORE] Race between end-of-task and completion iterator read lock release ## What changes were proposed in this pull request? When a TaskContext is not propagated properly to all child threads for the task, just like the reported cases in this issue, we fail to get to TID from TaskContext and that causes unable to release the lock and assertion failures. To resolve this, we have to explicitly pass the TID value to the `unlock` method. ## How was this patch tested? Add new failing regression test case in `RDDSuite`. Author: Xingbo Jiang <xingbo.jiang@databricks.com> Closes #18076 from jiangxb1987/completion-iterator. (cherry picked from commit d76633e3cad341b9efa23629f33c5ce90993d6d4) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 24 May 2017, 07:43:38 UTC
00dee39 [SPARK-20861][ML][PYTHON] Delegate looping over paramMaps to estimators Changes: pyspark.ml Estimators can take either a list of param maps or a dict of params. This change allows the CrossValidator and TrainValidationSplit Estimators to pass through lists of param maps to the underlying estimators so that those estimators can handle parallelization when appropriate (eg distributed hyper parameter tuning). Testing: Existing unit tests. Author: Bago Amirbekian <bago@databricks.com> Closes #18077 from MrBago/delegate_params. (cherry picked from commit 9434280cfd1db94dc9d52bb0ace8283e710e3124) Signed-off-by: Joseph K. Bradley <joseph@databricks.com> 24 May 2017, 03:56:12 UTC
d20c646 [SPARK-20857][SQL] Generic resolved hint node ## What changes were proposed in this pull request? This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes. ## How was this patch tested? Updated test cases. Author: Reynold Xin <rxin@databricks.com> Closes #18072 from rxin/SPARK-20857. (cherry picked from commit 0d589ba00b5d539fbfef5174221de046a70548cd) Signed-off-by: Reynold Xin <rxin@databricks.com> 23 May 2017, 16:45:08 UTC
dbb068f [MINOR][SPARKR][ML] Joint coefficients with intercept for SparkR linear SVM summary. ## What changes were proposed in this pull request? Joint coefficients with intercept for SparkR linear SVM summary. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #18035 from yanboliang/svm-r. (cherry picked from commit ad09e4ca045715d053a672c2ba23f598f06085d8) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 23 May 2017, 08:16:29 UTC
06c985c [SPARK-20399][SQL][FOLLOW-UP] Add a config to fallback string literal parsing consistent with old sql parser behavior ## What changes were proposed in this pull request? As srowen pointed in https://github.com/apache/spark/commit/609ba5f2b9fd89b1b9971d08f7cc680d202dbc7c#commitcomment-22221259, the previous tests are not proper. This follow-up is going to fix the tests. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18048 from viirya/SPARK-20399-follow-up. (cherry picked from commit 442287ae2993c24abb31fee6b5f6395724ca3dc7) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 23 May 2017, 08:09:55 UTC
5e9541a [SPARK-20727] Skip tests that use Hadoop utils on CRAN Windows ## What changes were proposed in this pull request? This change skips tests that use the Hadoop libraries while running on CRAN check with Windows as the operating system. This is to handle cases where the Hadoop winutils binaries are missing on the target system. The skipped tests consist of 1. Tests that save, load a model in MLlib 2. Tests that save, load CSV, JSON and Parquet files in SQL 3. Hive tests ## How was this patch tested? Tested by running on a local windows VM with HADOOP_HOME unset. Also testing with https://win-builder.r-project.org Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #17966 from shivaram/sparkr-windows-cran. (cherry picked from commit d06610f992ccf199928c0a71699fbf4c01705c31) Signed-off-by: Felix Cheung <felixcheung@apache.org> 23 May 2017, 06:04:34 UTC
ddc199e [SPARK-20815][SPARKR] NullPointerException in RPackageUtils#checkManifestForR ## What changes were proposed in this pull request? - Add a null check to RPackageUtils#checkManifestForR so that jars w/o manifests don't NPE. ## How was this patch tested? - Unit tests and manual tests. Author: James Shuster <jshuster@palantir.com> Closes #18040 from jrshust/feature/r-package-utils. (cherry picked from commit 4dbb63f0857a9cfb018cf49e3d1103cacc862ba2) Signed-off-by: Felix Cheung <felixcheung@apache.org> 23 May 2017, 04:41:23 UTC
d8328d8 [SPARK-20814][MESOS] Restore support for spark.executor.extraClassPath. Restore code that was removed as part of SPARK-17979, but instead of using the deprecated env variable name to propagate the class path, use a new one. Verified by running "./bin/spark-class o.a.s.executor.CoarseGrainedExecutorBackend" manually. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #18037 from vanzin/SPARK-20814. (cherry picked from commit df64fa79d678e354efe3969a6ba25d964128ce18) Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com> 22 May 2017, 19:34:24 UTC
2fd6138 [SPARK-20756][YARN] yarn-shuffle jar references unshaded guava and contains scala classes ## What changes were proposed in this pull request? This change ensures that all references to guava from within the yarn shuffle jar pointed to the shaded guava class already provided in the jar. Also, it explicitly excludes scala classes from being added to the jar. ## How was this patch tested? Ran unit tests on the module and they passed. javap now returns the expected result - reference to the shaded guava under `org/spark_project` (previously this was referring to `com.google...` ``` javap -cp common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar -c org/apache/spark/network/yarn/YarnShuffleService | grep Lists 57: invokestatic #138 // Method org/spark_project/guava/collect/Lists.newArrayList:()Ljava/util/ArrayList; ``` Guava is still shaded in the jar: ``` jar -tf common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar | grep guava | head META-INF/maven/com.google.guava/ META-INF/maven/com.google.guava/guava/ META-INF/maven/com.google.guava/guava/pom.properties META-INF/maven/com.google.guava/guava/pom.xml org/spark_project/guava/ org/spark_project/guava/annotations/ org/spark_project/guava/annotations/Beta.class org/spark_project/guava/annotations/GwtCompatible.class org/spark_project/guava/annotations/GwtIncompatible.class org/spark_project/guava/annotations/VisibleForTesting.class ``` (not sure if the above META-INF/* is a problem or not) I took this jar, deployed it on a yarn cluster with shuffle service enabled, and made sure the YARN node managers came up. An application with a shuffle was run and it succeeded. Author: Mark Grover <mark@apache.org> Closes #17990 from markgrover/spark-20756. (cherry picked from commit 36309110046a89d749a7c9746eaa16997de26922) Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com> 22 May 2017, 17:10:53 UTC
a0bf5c4 [SPARK-20764][ML][PYSPARK] Fix visibility discrepancy with numInstances and degreesOfFreedom in LR and GLR - Python version ## What changes were proposed in this pull request? SPARK-20097 exposed degreesOfFreedom in LinearRegressionSummary and numInstances in GeneralizedLinearRegressionSummary. Python API should be updated to reflect these changes. ## How was this patch tested? The existing UT Author: Peng <peng.meng@intel.com> Closes #18062 from mpjlu/spark-20764. (cherry picked from commit cfca01136bd7443c1d9daf8e8e256635eec20ddc) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 22 May 2017, 14:42:56 UTC
a575532 [SPARK-20831][SQL] Fix INSERT OVERWRITE data source tables with IF NOT EXISTS ### What changes were proposed in this pull request? Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query: ```SQL INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10 ``` we will get the following error: ``` unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;; 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true +- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430] +- Project [9 AS 9#423, 10 AS 10#424] +- OneRowRelation$ ``` This PR is to fix the issue to follow the behavior of Hive serde tables > INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition ### How was this patch tested? Modified an existing test case Author: gatorsmile <gatorsmile@gmail.com> Closes #18050 from gatorsmile/insertPartitionIfNotExists. (cherry picked from commit f3ed62a381897711d86fde27ab80bb70ed0b0513) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 22 May 2017, 14:25:19 UTC
81f63c8 [SPARK-20801] Record accurate size of blocks in MapStatus when it's above threshold. ## What changes were proposed in this pull request? Currently, when number of reduces is above 2000, HighlyCompressedMapStatus is used to store size of blocks. in HighlyCompressedMapStatus, only average size is stored for non empty blocks. Which is not good for memory control when we shuffle blocks. It makes sense to store the accurate size of block when it's above threshold. ## How was this patch tested? Added test in MapStatusSuite. Author: jinxing <jinxing6042@126.com> Closes #18031 from jinxing64/SPARK-20801. (cherry picked from commit 2597674bcc295c2e29c4cfc4a9a48938bd63bf9c) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 22 May 2017, 14:10:06 UTC
c4b16dc [SPARK-20813][WEB UI] Fixed Web UI executor page tab search by status not working ## What changes were proposed in this pull request? On status column of the table, I removed the condition that forced only the display value to take on values Active, Blacklisted and Dead. Before the removal, values used for sort and filter for that particular column was True and False. ## How was this patch tested? Tested with Active, Blacklisted and Dead present as current status. Author: John Lee <jlee2@yahoo-inc.com> Closes #18036 from yoonlee95/SPARK-20813. (cherry picked from commit aea73be1b436f5812dacc05c002c653f29e344de) Signed-off-by: Sean Owen <sowen@cloudera.com> 22 May 2017, 13:24:57 UTC
50dba30 [SPARK-20506][DOCS] Add HTML links to highlight list in MLlib guide for 2.2 Quick follow up to #17996 - forgot to add the HTML links to the relevant sections of the guide in the highlights list. ## How was this patch tested? Built docs locally and tested links. Author: Nick Pentreath <nickp@za.ibm.com> Closes #18043 from MLnick/SPARK-20506-2.2-migration-guide-2. (cherry picked from commit be846db48b226de2b0dfb5f87d059eda15ecf7cd) Signed-off-by: Nick Pentreath <nickp@za.ibm.com> 22 May 2017, 10:29:41 UTC
af1ff8b [SPARK-20687][MLLIB] mllib.Matrices.fromBreeze may crash when converting from Breeze sparse matrix ## What changes were proposed in this pull request? When two Breeze SparseMatrices are operated, the result matrix may contain provisional 0 values extra in rowIndices and data arrays. This causes an incoherence with the colPtrs data, but Breeze get away with this incoherence by keeping a counter of the valid data. In spark, when this matrices are converted to SparseMatrices, Sparks relies solely on rowIndices, data, and colPtrs, but these might be incorrect because of breeze internal hacks. Therefore, we need to slice both rowIndices and data, using their counter of active data This method is at least called by BlockMatrix when performing distributed block operations, causing exceptions on valid operations. See http://stackoverflow.com/questions/33528555/error-thrown-when-using-blockmatrix-add ## How was this patch tested? Added a test to MatricesSuite that verifies that the conversions are valid and that code doesn't crash. Originally the same code would crash on Spark. Bugfix for https://issues.apache.org/jira/browse/SPARK-20687 Author: Ignacio Bermudez <ignaciobermudez@gmail.com> Author: Ignacio Bermudez Corrales <icorrales@splunk.com> Closes #17940 from ghoto/bug-fix/SPARK-20687. (cherry picked from commit 06dda1d58f8670e996921e935d5f5402d664699e) Signed-off-by: Sean Owen <sowen@cloudera.com> 22 May 2017, 09:27:37 UTC
41d8d21 [SPARK-19089][SQL] Add support for nested sequences ## What changes were proposed in this pull request? Replaced specific sequence encoders with generic sequence encoder to enable nesting of sequences. Does not add support for nested arrays as that cannot be solved in this way. ## How was this patch tested? ```bash build/mvn -DskipTests clean package && dev/run-tests ``` Additionally in Spark shell: ``` scala> Seq(Seq(Seq(1))).toDS.collect() res0: Array[Seq[Seq[Int]]] = Array(List(List(1))) ``` Author: Michal Senkyr <mike.senkyr@gmail.com> Closes #18011 from michalsenkyr/dataset-seq-nested. (cherry picked from commit a2b3b67624ce7bbb29ddade03c1791d95e51869b) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 22 May 2017, 08:49:36 UTC
cfd1bf0 [SPARK-20792][SS] Support same timeout operations in mapGroupsWithState function in batch queries as in streaming queries ## What changes were proposed in this pull request? Currently, in the batch queries, timeout is disabled (i.e. GroupStateTimeout.NoTimeout) which means any GroupState.setTimeout*** operation would throw UnsupportedOperationException. This makes it weird when converting a streaming query into a batch query by changing the input DF from streaming to a batch DF. If the timeout was enabled and used, then the batch query will start throwing UnsupportedOperationException. This PR creates the dummy state in batch queries with the provided timeoutConf so that it behaves in the same way. The code has been refactored to make it obvious when the state is being created for a batch query or a streaming query. ## How was this patch tested? Additional tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #18024 from tdas/SPARK-20792. (cherry picked from commit 9d6661c829a4a82aae64ed0522c44e4c3d8f4f0b) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 21 May 2017, 20:07:32 UTC
3aad598 [SPARK-20781] the location of Dockerfile in docker.properties.templat is wrong [https://issues.apache.org/jira/browse/SPARK-20781](https://issues.apache.org/jira/browse/SPARK-20781) the location of Dockerfile in docker.properties.template should be "../external/docker/spark-mesos/Dockerfile" Author: liuzhaokun <liu.zhaokun@zte.com.cn> Closes #18013 from liu-zhaokun/dockerfile_location. (cherry picked from commit 749418d285461958a0f22ed355edafd87f1ee913) Signed-off-by: Sean Owen <sowen@cloudera.com> 19 May 2017, 19:47:37 UTC
4fcd52b [SPARK-20506][DOCS] 2.2 migration guide Update ML guide for migration `2.1` -> `2.2` and the previous version migration guide section. ## How was this patch tested? Build doc locally. Author: Nick Pentreath <nickp@za.ibm.com> Closes #17996 from MLnick/SPARK-20506-2.2-migration-guide. (cherry picked from commit b5d8d9ba17d62167cfbacd5f6188a8b4a5b8a2be) Signed-off-by: Nick Pentreath <nickp@za.ibm.com> 19 May 2017, 18:52:14 UTC
43f9fb7 [SPARK-20763][SQL] The function of `month` and `day` return the value which is not we expected. ## What changes were proposed in this pull request? spark-sql>select month("1582-09-28"); spark-sql>10 For this case, the expected result is 9, but it is 10. spark-sql>select day("1582-04-18"); spark-sql>28 For this case, the expected result is 18, but it is 28. when the date before "1582-10-04", the function of `month` and `day` return the value which is not we expected. ## How was this patch tested? unit tests Author: liuxian <liu.xian3@zte.com.cn> Closes #17997 from 10110346/wip_lx_0516. (cherry picked from commit ea3b1e352a605cd35cdee987d0e5eb8528ef1b45) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 19 May 2017, 17:25:29 UTC
001b82c [SPARK-20759] SCALA_VERSION in _config.yml should be consistent with pom.xml [https://issues.apache.org/jira/browse/SPARK-20759](https://issues.apache.org/jira/browse/SPARK-20759) SCALA_VERSION in _config.yml is 2.11.7, but 2.11.8 in pom.xml. So I think SCALA_VERSION in _config.yml should be consistent with pom.xml. Author: liuzhaokun <liu.zhaokun@zte.com.cn> Closes #17992 from liu-zhaokun/new. (cherry picked from commit dba2ca2c129b6d2597f1707e0315d4e238c40ed6) Signed-off-by: Sean Owen <sowen@cloudera.com> 19 May 2017, 14:26:47 UTC
939b953 [SPARK-20773][SQL] ParquetWriteSupport.writeFields is quadratic in number of fields Fix quadratic List indexing in ParquetWriteSupport. I noticed this function while profiling some code with today. It showed up as a significant factor in a table with twenty columns; with hundreds of columns, it could dominate any other function call. ## What changes were proposed in this pull request? The writeFields method iterates from 0 until number of fields, indexing into rootFieldWriters for each element. rootFieldWriters is a List, so indexing is a linear operation. The complexity of the writeFields method is thus quadratic in the number of fields. Solution: explicitly convert rootFieldWriters to Array (implicitly converted to WrappedArray) for constant-time indexing. ## How was this patch tested? This is a one-line change for performance reasons. Author: tpoterba <tpoterba@broadinstitute.org> Author: Tim Poterba <tpoterba@gmail.com> Closes #18005 from tpoterba/tpoterba-patch-1. (cherry picked from commit 3f2cd51ee06f2c6d735754e5440bc4b74f8dcbc8) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 19 May 2017, 12:17:24 UTC
2eed4c9 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter ## What changes were proposed in this pull request? GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior. Example of code generated before: ```scala /* 059 */ final UTF8String fieldName = value.getUTF8String(0); /* 060 */ if (value.isNullAt(0)) { /* 061 */ rowWriter1.setNullAt(0); /* 062 */ } else { /* 063 */ rowWriter1.write(0, fieldName); /* 064 */ } ``` Example of code generated now: ```scala /* 060 */ boolean isNull1 = value.isNullAt(0); /* 061 */ UTF8String value1 = isNull1 ? null : value.getUTF8String(0); /* 062 */ if (isNull1) { /* 063 */ rowWriter1.setNullAt(0); /* 064 */ } else { /* 065 */ rowWriter1.write(0, value1); /* 066 */ } ``` ## How was this patch tested? Adds GenerateUnsafeProjectionSuite. Author: Ala Luszczak <ala@databricks.com> Closes #18030 from ala/fix-generate-unsafe-projection. (cherry picked from commit ce8edb8bf4db5f82bcfeb11efbdf5229b0d25dfa) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 19 May 2017, 11:19:00 UTC
556ad01 [DSTREAM][DOC] Add documentation for kinesis retry configurations ## What changes were proposed in this pull request? The changes were merged as part of - https://github.com/apache/spark/pull/17467. The documentation was missed somewhere in the review iterations. Adding the documentation where it belongs. ## How was this patch tested? Docs. Not tested. cc budde , brkyvz Author: Yash Sharma <ysharma@atlassian.com> Closes #18028 from yssharma/ysharma/kinesis_retry_docs. (cherry picked from commit 92580bd0eae5dbf739573093cca1b12fd0c14049) Signed-off-by: Burak Yavuz <brkyvz@gmail.com> 18 May 2017, 18:24:44 UTC
8b0cb3a [SPARK-20364][SQL] Disable Parquet predicate pushdown for fields having dots in the names ## What changes were proposed in this pull request? This is an alternative workaround by simply avoiding the predicate pushdown for columns having dots in the names. This is an approach different with https://github.com/apache/spark/pull/17680. The downside of this PR is, literally it does not push down filters on the column having dots in Parquet files at all (both no record level and no rowgroup level) whereas the downside of the approach in that PR, it does not use the Parquet's API properly but in a hacky way to support this case. I assume we prefer a safe way here by using the Parquet API properly but this does close that PR as we are basically just avoiding here. This way looks a simple workaround and probably it is fine given the problem looks arguably rather corner cases (although it might end up with reading whole row groups under the hood but either looks not the best). Currently, if there are dots in the column name, predicate pushdown seems being failed in Parquet. **With dots** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("col.dots").write.parquet(path) spark.read.parquet(path).where("`col.dots` IS NOT NULL").show() ``` ``` +--------+ |col.dots| +--------+ +--------+ ``` **Without dots** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("coldots").write.parquet(path) spark.read.parquet(path).where("`coldots` IS NOT NULL").show() ``` ``` +-------+ |coldots| +-------+ | 1| +-------+ ``` **After** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("col.dots").write.parquet(path) spark.read.parquet(path).where("`col.dots` IS NOT NULL").show() ``` ``` +--------+ |col.dots| +--------+ | 1| +--------+ ``` ## How was this patch tested? Unit tests added in `ParquetFilterSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18000 from HyukjinKwon/SPARK-20364-workaround. (cherry picked from commit 8fb3d5c6da30922458091837eec17ccca502098a) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 18 May 2017, 17:52:32 UTC
db821fe [SPARK-20796] the location of start-master.sh in spark-standalone.md is wrong [https://issues.apache.org/jira/browse/SPARK-20796](https://issues.apache.org/jira/browse/SPARK-20796) the location of start-master.sh in spark-standalone.md should be "sbin/start-master.sh" rather than "bin/start-master.sh". Author: liuzhaokun <liu.zhaokun@zte.com.cn> Closes #18027 from liu-zhaokun/sbin. (cherry picked from commit 99452df44fb98c2721d427da4c97f549793615fe) Signed-off-by: Sean Owen <sowen@cloudera.com> 18 May 2017, 16:44:48 UTC
c708b14 [SPARK-20700][SQL] InferFiltersFromConstraints stackoverflows for query (v2) ## What changes were proposed in this pull request? In the previous approach we used `aliasMap` to link an `Attribute` to the expression with potentially the form `f(a, b)`, but we only searched the `expressions` and `children.expressions` for this, which is not enough when an `Alias` may lies deep in the logical plan. In that case, we can't generate the valid equivalent constraint classes and thus we fail at preventing the recursive deductions. We fix this problem by collecting all `Alias`s from the logical plan. ## How was this patch tested? No additional test case is added, but do modified one test case to cover this situation. Author: Xingbo Jiang <xingbo.jiang@databricks.com> Closes #18020 from jiangxb1987/inferConstrants. (cherry picked from commit b7aac15d566b048c20c2491fbf376b727f2eeb68) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 18 May 2017, 06:32:39 UTC
ba0117c [SPARK-20505][ML] Add docs and examples for ml.stat.Correlation and ml.stat.ChiSquareTest. ## What changes were proposed in this pull request? Add docs and examples for ```ml.stat.Correlation``` and ```ml.stat.ChiSquareTest```. ## How was this patch tested? Generate docs and run examples manually, successfully. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17994 from yanboliang/spark-20505. (cherry picked from commit 697a5e5517e32c5ef44c273e3b26662d0eb70f24) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 18 May 2017, 03:54:21 UTC
b8fa79c [SPARK-13747][CORE] Add ThreadUtils.awaitReady and disallow Await.ready ## What changes were proposed in this pull request? Add `ThreadUtils.awaitReady` similar to `ThreadUtils.awaitResult` and disallow `Await.ready`. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17763 from zsxwing/awaitready. (cherry picked from commit 324a904d8e80089d8865e4c7edaedb92ab2ec1b2) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 18 May 2017, 00:21:56 UTC
2db6101 [SPARK-20788][CORE] Fix the Executor task reaper's false alarm warning logs ## What changes were proposed in this pull request? Executor task reaper may fail to detect if a task is finished or not when a task is finishing but being killed at the same time. The fix is pretty easy, just flip the "finished" flag when a task is successful. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #18021 from zsxwing/SPARK-20788. (cherry picked from commit f8e0f0f47c15ddd646b9f295b91d6748583fe011) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 17 May 2017, 21:13:56 UTC
dac0b50 [SPARK-20769][DOC] Incorrect documentation for using Jupyter notebook ## What changes were proposed in this pull request? SPARK-13973 incorrectly removed the required PYSPARK_DRIVER_PYTHON_OPTS=notebook from documentation to use pyspark with Jupyter notebook. This patch corrects the documentation error. ## How was this patch tested? Tested invocation locally with ```bash PYSPARK_DRIVER_PYTHON=jupyter PYSPARK_DRIVER_PYTHON_OPTS=notebook ./bin/pyspark ``` Author: Andrew Ray <ray.andrew@gmail.com> Closes #18001 from aray/patch-1. (cherry picked from commit 1995417696a028f8a4fa7f706a77537c7182528d) Signed-off-by: Sean Owen <sowen@cloudera.com> 17 May 2017, 09:06:08 UTC
d42c67a [SPARK-20776] Fix perf. problems in JobProgressListener caused by TaskMetrics construction ## What changes were proposed in this pull request? In ``` ./bin/spark-shell --master=local[64] ``` I ran ``` sc.parallelize(1 to 100000, 100000).count() ``` and profiled the time spend in the LiveListenerBus event processing thread. I discovered that the majority of the time was being spent in `TaskMetrics.empty` calls in `JobProgressListener.onTaskStart`. It turns out that we can slightly refactor to remove the need to construct one empty instance per call, greatly improving the performance of this code. The performance gains here help to avoid an issue where listener events would be dropped because the JobProgressListener couldn't keep up with the throughput. **Before:** ![image](https://cloud.githubusercontent.com/assets/50748/26133095/95bcd42a-3a59-11e7-8051-a50550e447b8.png) **After:** ![image](https://cloud.githubusercontent.com/assets/50748/26133070/7935e148-3a59-11e7-8c2d-73d5aa5a2397.png) ## How was this patch tested? Benchmarks described above. Author: Josh Rosen <joshrosen@databricks.com> Closes #18008 from JoshRosen/nametoaccums-improvements. (cherry picked from commit 30e0557dbc134898ee65fe67d31054dcc8728576) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 17 May 2017, 05:04:48 UTC
7076ab4 [SPARK-20140][DSTREAM] Remove hardcoded kinesis retry wait and max retries ## What changes were proposed in this pull request? The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES. This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge. Following happens in a typical kinesis recovery : - kinesis throttles large number of requests while recovering - retries in case of throttling are not able to recover due to the small wait period - kinesis throttles per second, the wait period should be configurable for recovery The patch picks the spark kinesis configs from: - spark.streaming.kinesis.retry.wait.time - spark.streaming.kinesis.retry.max.attempts Jira : https://issues.apache.org/jira/browse/SPARK-20140 ## How was this patch tested? Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling. Author: Yash Sharma <ysharma@atlassian.com> Closes #17467 from yssharma/ysharma/spark-kinesis-retries. (cherry picked from commit 38f4e8692ce3b6cbcfe0c1aff9b5e662f7a308b7) Signed-off-by: Burak Yavuz <brkyvz@gmail.com> 16 May 2017, 22:08:46 UTC
75e5ea2 [SPARK-20529][CORE] Allow worker and master work with a proxy server ## What changes were proposed in this pull request? In the current codes, when worker connects to master, master will send its address to the worker. Then worker will save this address and use it to reconnect in case of failure. However, sometimes, this address is not correct. If there is a proxy between master and worker, the address master sent is not the address of proxy. In this PR, the master address used by the worker will be sent to the master, then master just replies this address back, worker will use this address to reconnect in case of failure. In other words, the worker will use the config master address set in the worker side if possible rather than the master address set in the master side. There is still one potential issue though. When a master is restarted or takes over leadership, the work will use the address sent from the master to connect. If there is still a proxy between master and worker, the address may be wrong. However, there is no way to figure it out just in the worker. ## How was this patch tested? The new added unit test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #17821 from zsxwing/SPARK-20529. (cherry picked from commit 9150bca47e4b8782e20441386d3d225eb5f2f404) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 16 May 2017, 17:38:12 UTC
ee0d2af [SPARK-20677][MLLIB][ML] Follow-up to ALS recommend-all performance PRs Small clean ups from #17742 and #17845. ## How was this patch tested? Existing unit tests. Author: Nick Pentreath <nickp@za.ibm.com> Closes #17919 from MLnick/SPARK-20677-als-perf-followup. (cherry picked from commit 25b4f41d239ac67402566c0254a893e2e58ae7d8) Signed-off-by: Nick Pentreath <nickp@za.ibm.com> 16 May 2017, 08:59:52 UTC
b8d37ac [SPARK-20553][ML][PYSPARK] Update ALS examples with recommend-all methods Update ALS examples illustrating use of "recommendForAllX" methods. ## How was this patch tested? Built and ran examples locally Author: Nick Pentreath <nickp@za.ibm.com> Closes #17950 from MLnick/SPARK-20553-update-als-examples. (cherry picked from commit 6af7b43b34942c662122e3905b0724b2dd40a63f) Signed-off-by: Nick Pentreath <nickp@za.ibm.com> 16 May 2017, 08:55:01 UTC
57c87cf [SPARK-20501][ML] ML 2.2 QA: New Scala APIs, docs ## What changes were proposed in this pull request? Review new Scala APIs introduced in 2.2. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17934 from yanboliang/spark-20501. (cherry picked from commit dbe81633a766c4dc68a0a27063e5dfde0f5690af) Signed-off-by: Joseph K. Bradley <joseph@databricks.com> 16 May 2017, 04:22:26 UTC
a869e8b [SPARK-20707][ML] ML deprecated APIs should be removed in major release. ## What changes were proposed in this pull request? Before 2.2, MLlib keep to remove APIs deprecated in last feature/minor release. But from Spark 2.2, we decide to remove deprecated APIs in a major release, so we need to change corresponding annotations to tell users those will be removed in 3.0. Meanwhile, this fixed bugs in ML documents. The original ML docs can't show deprecated annotations in ```MLWriter``` and ```MLReader``` related class, we correct it in this PR. Before: ![image](https://cloud.githubusercontent.com/assets/1962026/25939889/f8c55f20-3666-11e7-9fa2-0605bfb3ed06.png) After: ![image](https://cloud.githubusercontent.com/assets/1962026/25939870/e9b0d5be-3666-11e7-9765-5e04885e4b32.png) ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17946 from yanboliang/spark-20707. (cherry picked from commit d4022d49514cc1f8ffc5bfe243186ec3748df475) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 16 May 2017, 02:08:35 UTC
10e599f [SPARK-20588][SQL] Cache TimeZone instances. ## What changes were proposed in this pull request? Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck. This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site. This pr makes a cache of the generated TimeZone instances to avoid the synchronization. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@databricks.com> Closes #17933 from ueshin/issues/SPARK-20588. (cherry picked from commit c8c878a4166415728f6e940504766a099a2f6744) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 15 May 2017, 23:52:31 UTC
e84e9dd [SPARK-20735][SQL][TEST] Enable cross join in TPCDSQueryBenchmark ## What changes were proposed in this pull request? Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_". This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`. ## How was this patch tested? Manual. (Run TPCDSQueryBenchmark) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #17977 from dongjoon-hyun/SPARK-20735. (cherry picked from commit bbd163d589e7503c5cb150d934e7565b18a908f2) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 15 May 2017, 18:24:40 UTC
a79a120 [SPARK-20717][SS] Minor tweaks to the MapGroupsWithState behavior ## What changes were proposed in this pull request? Timeout and state data are two independent entities and should be settable independently. Therefore, in the same call of the user-defined function, one should be able to set the timeout before initializing the state and also after removing the state. Whether timeouts can be set or not, should not depend on the current state, and vice versa. However, a limitation of the current implementation is that state cannot be null while timeout is set. This is checked lazily after the function call has completed. ## How was this patch tested? - Updated existing unit tests that test the behavior of GroupState.setTimeout*** wrt to the current state - Added new tests that verify the disallowed cases where state is undefined but timeout is set. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17957 from tdas/SPARK-20717. (cherry picked from commit 499ba2cb47efd6a860e74e6995412408efc5238d) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 15 May 2017, 17:48:17 UTC
82ae1f0 [SPARK-20716][SS] StateStore.abort() should not throw exceptions ## What changes were proposed in this pull request? StateStore.abort() should do a best effort attempt to clean up temporary resources. It should not throw errors, especially because its called in a TaskCompletionListener, because this error could hide previous real errors in the task. ## How was this patch tested? No unit test. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17958 from tdas/SPARK-20716. (cherry picked from commit 271175e2bd0f7887a068db92de73eff60f5ef2b2) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 15 May 2017, 17:46:45 UTC
0bd918f [SPARK-12837][SPARK-20666][CORE][FOLLOWUP] getting name should not fail if accumulator is garbage collected ## What changes were proposed in this pull request? After https://github.com/apache/spark/pull/17596 , we do not send internal accumulator name to executor side anymore, and always look up the accumulator name in `AccumulatorContext`. This cause a regression if the accumulator is already garbage collected, this PR fixes this by still sending accumulator name for `SQLMetrics`. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #17931 from cloud-fan/bug. (cherry picked from commit e1aaab1e277b1b07c26acea75ade78e39bdac209) Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com> 15 May 2017, 16:22:24 UTC
708f68c [SPARK-20669][ML] LoR.family and LDA.optimizer should be case insensitive ## What changes were proposed in this pull request? make param `family` in LoR and `optimizer` in LDA case insensitive ## How was this patch tested? updated tests yanboliang Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #17910 from zhengruifeng/lr_family_lowercase. (cherry picked from commit 9970aa0962ec253a6e838aea26a627689dc5b011) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 15 May 2017, 15:21:57 UTC
cb64064 [SPARK-20705][WEB-UI] The sort function can not be used in the master page when you use Firefox or Google Chrome. ## What changes were proposed in this pull request? When you open the master page, when you use Firefox or Google Chrom, the console of Firefox or Google Chrome is wrong. But The IE is no problem. e.g. ![error](https://cloud.githubusercontent.com/assets/26266482/25946143/74467a5c-367c-11e7-8f9f-d3585b1aea88.png) My Firefox version is 48.0.2. My Google Chrome version is 49.0.2623.75 m. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: guoxiaolong <guo.xiaolong1@zte.com.cn> Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn> Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn> Closes #17952 from guoxiaolongzte/SPARK-20705. (cherry picked from commit 99d5799927301e7dfceb9405e2829af3433f104b) Signed-off-by: Sean Owen <sowen@cloudera.com> 15 May 2017, 06:51:58 UTC
5842eec [SPARK-20725][SQL] partial aggregate should behave correctly for sameResult ## What changes were proposed in this pull request? For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them. ## How was this patch tested? a new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17964 from cloud-fan/tmp. (cherry picked from commit 1283c3d11af6d55eaf0e40d6df09dc6bcc198322) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 13 May 2017, 19:09:16 UTC
02bf554 [SPARK-18772][SQL] Avoid unnecessary conversion try for special floats in JSON ## What changes were proposed in this pull request? This PR is based on https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772 This avoids additional conversion try with `toFloat` and `toDouble`. For avoiding additional conversions, please refer the codes below: **Before** ```scala scala> import org.apache.spark.sql.types._ import org.apache.spark.sql.types._ scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show() 17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2) java.lang.NumberFormatException: For input string: "nan" ... ``` **After** ```scala scala> import org.apache.spark.sql.types._ import org.apache.spark.sql.types._ scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show() 17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.RuntimeException: Cannot parse nan as DoubleType. ... ``` ## How was this patch tested? Unit tests added in `JsonSuite`. Closes #16199 Author: hyukjinkwon <gurwls223@gmail.com> Author: Nathan Howell <nhowell@godaddy.com> Closes #17956 from HyukjinKwon/SPARK-18772. (cherry picked from commit 3f98375d8a0f3d48041042d356915a5c298b1c8e) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 13 May 2017, 12:56:30 UTC
d99165b [SPARK-20594][SQL] The staging directory should be a child directory starts with "." to avoid being deleted if we set hive.exec.stagingdir under the table directory. JIRA Issue: https://issues.apache.org/jira/browse/SPARK-20594 ## What changes were proposed in this pull request? The staging directory should be a child directory starts with "." to avoid being deleted before moving staging directory to table directory if we set hive.exec.stagingdir under the table directory. ## How was this patch tested? Added unit tests Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #17858 from zuotingbing/spark-stagingdir. (cherry picked from commit e3d2022e4b73c5bc352eb001ed334d91ce36109c) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 12 May 2017, 18:24:17 UTC
f142469 [SPARK-20714][SS] Fix match error when watermark is set with timeout = no timeout / processing timeout ## What changes were proposed in this pull request? When watermark is set, and timeout conf is NoTimeout or ProcessingTimeTimeout (both do not need the watermark), the query fails at runtime with the following exception. ``` MatchException: Some(org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate1a9b798e) (of class scala.Some) org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:120) org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:116) org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:70) org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:65) org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:64) ``` The match did not correctly handle cases where watermark was defined by the timeout was different from EventTimeTimeout. ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17954 from tdas/SPARK-20714. (cherry picked from commit 0d3a63193c691ece88bb256d04156258a1c03a81) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 12 May 2017, 17:49:58 UTC
7123ec8 [SPARK-20702][CORE] TaskContextImpl.markTaskCompleted should not hide the original error ## What changes were proposed in this pull request? This PR adds an `error` parameter to `TaskContextImpl.markTaskCompleted` to propagate the original error. It also fixes an issue that `TaskCompletionListenerException.getMessage` doesn't include `previousError`. ## How was this patch tested? New unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #17942 from zsxwing/SPARK-20702. (cherry picked from commit 7d6ff39106938fa4bbb68b3d5114b93a4d332c5c) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 12 May 2017, 17:46:54 UTC
eed3a5a [SPARK-20710][SQL] Support aliases in CUBE/ROLLUP/GROUPING SETS ## What changes were proposed in this pull request? This pr added `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191). ## How was this patch tested? Added tests in `SQLQueryTestSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17948 from maropu/SPARK-20710. (cherry picked from commit 92ea7fd7b6cd4641b2f02b97105835029ddadc5f) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 12:48:42 UTC
18725f5 [SPARK-20718][SQL][FOLLOWUP] Fix canonicalization for HiveTableScanExec ## What changes were proposed in this pull request? Fix canonicalization for different filter orders in `HiveTableScanExec`. ## How was this patch tested? Added a new test case. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17962 from wzhfy/canonicalizeHiveTableScanExec. (cherry picked from commit 54b4f2ad43c0ad333a3751a7f10da711b94677a0) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 12:44:10 UTC
10b7244 [SPARK-17424] Fix unsound substitution bug in ScalaReflection. ## What changes were proposed in this pull request? This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters. ## How was this patch tested? This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully. Author: Ryan Blue <blue@apache.org> Closes #15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution. (cherry picked from commit b23693390781a99ff9248ea07a22e68884ffc747) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 12:38:51 UTC
21764f8 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls ## What changes were proposed in this pull request? Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17949 from srowen/SPARK-20554. (cherry picked from commit fc8a2b6ee6f1041345f9ce9701fac496c3c3b1e6) Signed-off-by: Sean Owen <sowen@cloudera.com> 12 May 2017, 08:55:14 UTC
c1e5ac2 [SPARK-20704][SPARKR] change CRAN test to run single thread ## What changes were proposed in this pull request? - [x] need to test by running R CMD check --as-cran - [x] sanity check vignettes ## How was this patch tested? Jenkins Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17945 from felixcheung/rchangesforpackage. (cherry picked from commit 888b84abe8d3fd36c5c2226aeb9e202029936f94) Signed-off-by: Felix Cheung <felixcheung@apache.org> 12 May 2017, 06:10:19 UTC
a8d981d [SPARK-20718][SQL] FileSourceScanExec with different filter orders should be the same after canonicalization ## What changes were proposed in this pull request? Since `constraints` in `QueryPlan` is a set, the order of filters can differ. Usually this is ok because of canonicalization. However, in `FileSourceScanExec`, its data filters and partition filters are sequences, and their orders are not canonicalized. So `def sameResult` returns different results for different orders of data/partition filters. This leads to, e.g. different decision for `ReuseExchange`, and thus results in unstable performance. ## How was this patch tested? Added a new test for `FileSourceScanExec.sameResult`. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17959 from wzhfy/canonicalizeFileSourceScanExec. (cherry picked from commit c8da5356000c8e4ff9141e4a2892ebe0b9641d63) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 05:43:04 UTC
2cac317 [SPARK-20665][SQL] Bround" and "Round" function return NULL ## What changes were proposed in this pull request? spark-sql>select bround(12.3, 2); spark-sql>NULL For this case, the expected result is 12.3, but it is null. So ,when the second parameter is bigger than "decimal.scala", the result is not we expected. "round" function has the same problem. This PR can solve the problem for both of them. ## How was this patch tested? unit test cases in MathExpressionsSuite and MathFunctionsSuite Author: liuxian <liu.xian3@zte.com.cn> Closes #17906 from 10110346/wip_lx_0509. (cherry picked from commit 2b36eb696f6c738e1328582630755aaac4293460) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 03:39:02 UTC
3d1908f [SPARK-20399][SQL] Add a config to fallback string literal parsing consistent with old sql parser behavior ## What changes were proposed in this pull request? The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string. The following codes can reproduce it: val data = Seq("\u0020\u0021\u0023", "abc") val df = data.toDF() // 1st usage: works in 1.6 // Let parser parse pattern string val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'") // 2nd usage: works in 1.6, 2.x // Call Column.rlike so the pattern string is a literal which doesn't go through parser val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$")) // In 2.x, we need add backslashes to make regex pattern parsed correctly val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'") Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17887 from viirya/add-config-fallback-string-parsing. (cherry picked from commit 609ba5f2b9fd89b1b9971d08f7cc680d202dbc7c) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 12 May 2017, 03:15:26 UTC
5844151 [SPARK-20600][SS] KafkaRelation should be pretty printed in web UI ## What changes were proposed in this pull request? User-friendly name of `KafkaRelation` in web UI (under Details for Query). ### Before <img width="516" alt="spark-20600-before" src="https://cloud.githubusercontent.com/assets/62313/25841955/74479ac6-34a2-11e7-87fb-d9f62a1356a7.png"> ### After <img width="439" alt="spark-20600-after" src="https://cloud.githubusercontent.com/assets/62313/25841829/f5335630-34a1-11e7-85a4-afe9b66d73c8.png"> ## How was this patch tested? Local build ``` ./bin/spark-shell --jars ~/.m2/repository/org/apache/spark/spark-sql-kafka-0-10_2.11/2.3.0-SNAPSHOT/spark-sql-kafka-0-10_2.11-2.3.0-SNAPSHOT.jar --packages org.apache.kafka:kafka-clients:0.10.0.1 ``` Author: Jacek Laskowski <jacek@japila.pl> Closes #17917 from jaceklaskowski/SPARK-20600-KafkaRelation-webUI. (cherry picked from commit 7144b51809aa99ac076786c369389e2330142beb) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 11 May 2017, 17:55:31 UTC
dd9e3b2 [SPARK-20569][SQL] RuntimeReplaceable functions should not take extra parameters ## What changes were proposed in this pull request? `RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17876 from cloud-fan/minor. (cherry picked from commit b4c99f43690f8cfba414af90fa2b3998a510bba8) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 11 May 2017, 07:41:35 UTC
80a57fa [SPARK-20606][ML] Revert "[] ML 2.2 QA: Remove deprecated methods for ML" This reverts commit b8733e0ad9f5a700f385e210450fd2c10137293e. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17944 from yanboliang/spark-20606-revert. (cherry picked from commit 0698e6c88ca11fdfd6e5498cab784cf6dbcdfacb) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 11 May 2017, 06:48:26 UTC
3eb0ee0 [SPARK-20685] Fix BatchPythonEvaluation bug in case of single UDF w/ repeated arg. ## What changes were proposed in this pull request? There's a latent corner-case bug in PySpark UDF evaluation where executing a `BatchPythonEvaluation` with a single multi-argument UDF where _at least one argument value is repeated_ will crash at execution with a confusing error. This problem was introduced in #12057: the code there has a fast path for handling a "batch UDF evaluation consisting of a single Python UDF", but that branch incorrectly assumes that a single UDF won't have repeated arguments and therefore skips the code for unpacking arguments from the input row (whose schema may not necessarily match the UDF inputs due to de-duplication of repeated arguments which occurred in the JVM before sending UDF inputs to Python). This fix here is simply to remove this special-casing: it turns out that the code in the "multiple UDFs" branch just so happens to work for the single-UDF case because Python treats `(x)` as equivalent to `x`, not as a single-argument tuple. ## How was this patch tested? New regression test in `pyspark.python.sql.tests` module (tested and confirmed that it fails before my fix). Author: Josh Rosen <joshrosen@databricks.com> Closes #17927 from JoshRosen/SPARK-20685. (cherry picked from commit 8ddbc431d8b21d5ee57d3d209a4f25e301f15283) Signed-off-by: Xiao Li <gatorsmile@gmail.com> 10 May 2017, 23:51:05 UTC
86cef4d [SPARK-19447] Remove remaining references to generated rows metric ## What changes were proposed in this pull request? https://github.com/apache/spark/commit/b486ffc86d8ad6c303321dcf8514afee723f61f8 left behind references to "number of generated rows" metrics, that should have been removed. ## How was this patch tested? Existing unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #17939 from ala/SPARK-19447-fix. (cherry picked from commit 5c2c4dcce529d228a97ede0386b95213ea0e1da5) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 10 May 2017, 15:41:14 UTC
358516d [MINOR][BUILD] Fix lint-java breaks. ## What changes were proposed in this pull request? This PR proposes to fix the lint-breaks as below: ``` [ERROR] src/main/java/org/apache/spark/unsafe/Platform.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed. [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[45,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[62,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[78,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[92,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[102,25] (naming) MethodName: Method name 'Once' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.api.java.JavaDStream. ``` after: ``` dev/lint-java Checkstyle checks passed. ``` [Test Result](https://travis-ci.org/ConeyLiu/spark/jobs/229666169) ## How was this patch tested? Travis CI Author: Xianyang Liu <xianyang.liu@intel.com> Closes #17890 from ConeyLiu/codestyle. (cherry picked from commit fcb88f9211e39c705073db5300c96ceeb3f227d7) Signed-off-by: Sean Owen <sowen@cloudera.com> 10 May 2017, 12:56:42 UTC
5f6029c [SPARK-20678][SQL] Ndv for columns not in filter condition should also be updated ## What changes were proposed in this pull request? In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not. This pr also fixes the inconsistency of rounding mode for ndv and rowCount. ## How was this patch tested? Added new tests. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17918 from wzhfy/scaleDownNdvAfterFilter. (cherry picked from commit 76e4a5566b1e9579632e03440cecd04dd142bc44) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 May 2017, 11:43:05 UTC
0851b6c [SPARK-20688][SQL] correctly check analysis for scalar sub-queries ## What changes were proposed in this pull request? In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17930 from cloud-fan/tmp. (cherry picked from commit 789bdbe3d0d9558043872161bdfa148ec021a849) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 May 2017, 11:30:26 UTC
7597a52 [SPARK-20630][WEB UI] Fixed column visibility in Executor Tab ## What changes were proposed in this pull request? #14617 added new columns to the executor table causing the visibility checks for the logs and threadDump columns to toggle the wrong columns since they used hard-coded column numbers. I've updated the checks to use column names instead of numbers so future updates don't accidentally break this again. Note: This will also need to be back ported into 2.2 since #14617 was merged there ## How was this patch tested? Manually tested Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #17904 from ajbozarth/spark20630. (cherry picked from commit ca4625e0e58df7f02346470d22a9478d9640709d) Signed-off-by: Sean Owen <sowen@cloudera.com> 10 May 2017, 09:20:19 UTC
3ed2f4d [SPARK-20631][PYTHON][ML] LogisticRegression._checkThresholdConsistency should use values not Params ## What changes were proposed in this pull request? - Replace `getParam` calls with `getOrDefault` calls. - Fix exception message to avoid unintended `TypeError`. - Add unit tests ## How was this patch tested? New unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #17891 from zero323/SPARK-20631. (cherry picked from commit 804949c6bf00b8e26c39d48bbcc4d0470ee84e47) Signed-off-by: Yanbo Liang <ybliang8@gmail.com> 10 May 2017, 08:58:08 UTC
ef50a95 [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping ## What changes were proposed in this pull request? The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. ## How was this patch tested? - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes #17929 from JoshRosen/fix-PropagateEmptyRelation. (cherry picked from commit a90c5cd8226146a58362732171b92cb99a7bc4c7) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 May 2017, 06:36:54 UTC
7b6f3a1 [SPARK-20590][SQL] Use Spark internal datasource if multiples are found for the same shorten name ## What changes were proposed in this pull request? One of the common usability problems around reading data in spark (particularly CSV) is that there can often be a conflict between different readers in the classpath. As an example, if someone launches a 2.x spark shell with the spark-csv package in the classpath, Spark currently fails in an extremely unfriendly way (see databricks/spark-csv#367): ```bash ./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0 scala> val df = spark.read.csv("/foo/bar.csv") java.lang.RuntimeException: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), please specify the fully qualified class name. at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:574) at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:85) at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:85) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:295) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178) at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:533) at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:412) ... 48 elided ``` This PR proposes a simple way of fixing this error by picking up the internal datasource if there is single (the datasource that has "org.apache.spark" prefix). ```scala scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc") 17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat). ``` ```scala scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc") 17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat). ``` ## How was this patch tested? Manually tested as below: ```bash ./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0 ``` ```scala spark.sparkContext.setLogLevel("WARN") ``` **positive cases**: ```scala scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc") 17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat). ``` ```scala scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc") 17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat). ``` (newlines were inserted for readability). ```scala scala> spark.range(1).write.format("com.databricks.spark.csv").mode("overwrite").save("/tmp/abc") ``` ```scala scala> spark.range(1).write.format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat").mode("overwrite").save("/tmp/abc") ``` **negative cases**: ```scala scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelation").save("/tmp/abc") java.lang.InstantiationException: com.databricks.spark.csv.CsvRelation ... ``` ```scala scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelatio").save("/tmp/abc") java.lang.ClassNotFoundException: Failed to find data source: com.databricks.spark.csv.CsvRelatio. Please find packages at http://spark.apache.org/third-party-projects.html ... ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #17916 from HyukjinKwon/datasource-detect. (cherry picked from commit 3d2131ab4ddead29601fb3c597b798202ac25fdd) Signed-off-by: Wenchen Fan <wenchen@databricks.com> 10 May 2017, 05:45:07 UTC
6a996b3 [SPARK-17685][SQL] Make SortMergeJoinExec's currentVars is null when calling createJoinKey ## What changes were proposed in this pull request? The following SQL query cause `IndexOutOfBoundsException` issue when `LIMIT > 1310720`: ```sql CREATE TABLE tab1(int int, int2 int, str string); CREATE TABLE tab2(int int, int2 int, str string); INSERT INTO tab1 values(1,1,'str'); INSERT INTO tab1 values(2,2,'str'); INSERT INTO tab2 values(1,1,'str'); INSERT INTO tab2 values(2,3,'str'); SELECT count(*) FROM ( SELECT t1.int, t2.int2 FROM (SELECT * FROM tab1 LIMIT 1310721) t1 INNER JOIN (SELECT * FROM tab2 LIMIT 1310721) t2 ON (t1.int = t2.int AND t1.int2 = t2.int2) ) t; ``` This pull request fix this issue. ## How was this patch tested? unit tests Author: Yuming Wang <wgyumg@gmail.com> Closes #17920 from wangyum/SPARK-17685. (cherry picked from commit 771abeb46f637592aba2e63db2ed05b6cabfd0be) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 10 May 2017, 02:45:09 UTC
7600a7a [SPARK-20373][SQL][SS] Batch queries with 'Dataset/DataFrame.withWatermark()` does not execute ## What changes were proposed in this pull request? Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan. The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way. Changes: - In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query. Depends upon: - [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly. Others: - A typo fix in example. ## How was this patch tested? add new unit test. Author: uncleGen <hustyugm@gmail.com> Closes #17896 from uncleGen/SPARK-20373. (cherry picked from commit c0189abc7c6ddbecc1832d2ff0cfc5546a010b60) Signed-off-by: Shixiong Zhu <shixiong@databricks.com> 09 May 2017, 22:08:38 UTC
d191b96 Revert "[SPARK-20311][SQL] Support aliases for table value functions" This reverts commit 714811d0b5bcb5d47c39782ff74f898d276ecc59. 09 May 2017, 21:49:02 UTC
9e8d23b [SPARK-20627][PYSPARK] Drop the hadoop distirbution name from the Python version ## What changes were proposed in this pull request? Drop the hadoop distirbution name from the Python version (PEP440 - https://www.python.org/dev/peps/pep-0440/). We've been using the local version string to disambiguate between different hadoop versions packaged with PySpark, but PEP0440 states that local versions should not be used when publishing up-stream. Since we no longer make PySpark pip packages for different hadoop versions, we can simply drop the hadoop information. If at a later point we need to start publishing different hadoop versions we can look at make different packages or similar. ## How was this patch tested? Ran `make-distribution` locally Author: Holden Karau <holden@us.ibm.com> Closes #17885 from holdenk/SPARK-20627-remove-pip-local-version-string. (cherry picked from commit 1b85bcd9298cf84dd746fe8e91ab0b0df69ef17e) Signed-off-by: Holden Karau <holden@us.ibm.com> 09 May 2017, 18:26:00 UTC
c7bd909 [SPARK-19876][BUILD] Move Trigger.java to java source hierarchy ## What changes were proposed in this pull request? Simply moves `Trigger.java` to `src/main/java` from `src/main/scala` See https://github.com/apache/spark/pull/17219 ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17921 from srowen/SPARK-19876.2. (cherry picked from commit 25ee816e090c42f0e35be2d2cb0f8ec60726317c) Signed-off-by: Herman van Hovell <hvanhovell@databricks.com> 09 May 2017, 17:22:32 UTC
back to top