## What changes were proposed in this pull request?
Since Spark 2.2.0 ([SPARK-19678](https://issues.apache.org/jira/browse/SPARK-19678)), the below SQL changed from `broadcast join` to `sort merge join`:
```sql
-- small external table with incorrect statistics
CREATE EXTERNAL TABLE t1(c1 int)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
WITH SERDEPROPERTIES (
'serialization.format' = '1'
)
STORED AS
INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION 'file:///tmp/t1'
TBLPROPERTIES (
'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1'
);
-- big table
CREATE TABLE t2 (c1 int)
LOCATION 'file:///tmp/t2'
TBLPROPERTIES (
'rawDataSize'='23437737', 'numFiles'='12222', 'totalSize'='333442230', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='443442223'
);
explain SELECT t1.c1 FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1;
```
This pr add a test case for this behavior change.
## How was this patch tested?
unit tests
Closes#24003 from wangyum/SPARK-19678.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This pr adds 2 maven properties to help us upgrade the built-in Hive.
| Property Name | Default | In future |
| ------ | ------ | ------ |
| hive.classifier | (none) | core |
| hive.parquet.group | com.twitter | org.apache.parquet |
## How was this patch tested?
existing tests
Closes#23996 from wangyum/add_2_maven_properties.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
`CodeGenerator.updateAndGetCompilationStats` throws an unsupported exception for empty code size statistics. This pr added code to check if it is empty or not.
## How was this patch tested?
Pass Jenkins.
Closes#23947 from maropu/SPARK-21871-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
This is a very minor pr to include the usage example to generate output for single test in SQLQueryTestSuite. I tried to deduce it from the existing example and ran into a scenario
where sbt is simply looping to run the same test over and over again. Here is the example
of running a single test.
```
build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql"
```
I tried to generate the output for a single test by prepending `SPARK_GENERATE_GOLDEN_FILES=1` like following
```
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "~sql/test-only *SQLQueryTestSuite -- -z describe.sql"
```
In this case i found that sbt is looping trying to run describe.sql over and over again as we are running the test in on continuous mode (because of `~` prefix ) where it detects a change in
the generated result file which in turn triggers a build and test. I have included an example where
we don't run it in continuous mode when generating the output. Hopefully it saves other developers some time.
## How was this patch tested?
Verified manually in my dev setup.
Closes#23995 from dilipbiswal/dkb_sqlquerytest_usage.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
While I am migrating other data sources, I find that we should abstract the logic that:
1. converting safe `InternalRow`s into `UnsafeRow`s
2. appending partition values to the end of the result row if existed
This PR proposes to support handling partition values in file source v2 abstraction by adding a util class `PartitionReaderWithPartitionValues`.
## How was this patch tested?
Existing unit tests
Closes#23987 from gengliangwang/SPARK-27049.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This pr fix `NoSuchFieldError` when reading Hive materialized views from Hive 2.3.4.
How to reproduce:
Hive side:
```sql
CREATE TABLE materialized_view_tbl (key INT);
CREATE MATERIALIZED VIEW view_1 DISABLE REWRITE AS SELECT * FROM materialized_view_tbl;
```
Spark side:
```java
bin/spark-sql --conf spark.sql.hive.metastore.version=2.3.4 --conf spark.sql.hive.metastore.jars=maven
spark-sql> select * from view_1;
19/03/05 19:55:37 ERROR SparkSQLDriver: Failed in [select * from view_1]
java.lang.NoSuchFieldError: INDEX_TABLE
at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:438)
at scala.Option.map(Option.scala:163)
at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:370)
at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:277)
at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:215)
at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:214)
at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:260)
at org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:368)
```
## How was this patch tested?
unit tests
Closes#23984 from wangyum/SPARK-24360.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
I would like to refactor `limit.scala` slightly and introduce common trait `LimitExec` for `CollectLimitExec` and `BaseLimitExec` (`LocalLimitExec` and `GlobalLimitExec`). This will allow to distinguish those operators from others, and to get the `limit` value without casting to concrete class.
## How was this patch tested?
by existing test suites.
Closes#23976 from MaxGekk/limit-exec.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently, when the event reordering happens, especially onJobStart event come after onExecutionEnd event, SQL page in the UI displays weirdly.(for eg:test mentioned in JIRA and also this issue randomly occurs when the TPCDS query fails due to broadcast timeout etc.)
The reason is that, In the SQLAppstatusListener, we remove the liveExecutions entry once the execution ends. So, if a jobStart event come after that, then we create a new liveExecution entry corresponding to the execId. Eventually this will overwrite the kvstore and UI displays confusing entries.
## How was this patch tested?
Added UT, Also manually tested with the eventLog, provided in the jira, of the failed query.
Before fix:
![screenshot from 2019-03-03 03-05-52](https://user-images.githubusercontent.com/23054875/53687929-53e2b800-3d61-11e9-9dca-620fa41e605c.png)
After fix:
![screenshot from 2019-03-03 02-40-18](https://user-images.githubusercontent.com/23054875/53687928-4f1e0400-3d61-11e9-86aa-584646ac68f9.png)
Closes#23939 from shahidki31/SPARK-27019.
Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Before dropping database refresh the tables of that database, so as to refresh all cached entries associated with those tables.
We follow the same when dropping a table.
## How was this patch tested?
UT is added
Closes#23905 from Udbhav30/SPARK-24669.
Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
In the PR, I propose to replace `System.currentTimeMillis()` by `Instant.now()` in the `CurrentTimestamp` expression. `Instant.now()` uses the best available clock in the system to take current time. See [JDK-8068730](https://bugs.openjdk.java.net/browse/JDK-8068730) for more details. In JDK8, `Instant.now()` provides results with millisecond resolution but starting from JDK9 resolution of results is increased up to microseconds.
## How was this patch tested?
The changes were tested by `DateTimeUtilsSuite` and by `DateFunctionsSuite`.
Closes#23945 from MaxGekk/current-time.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Removed unnecessary conversion of microseconds in `DateTimeUtils.timestampToString` to `java.sql.Timestamp` which aims to output fraction of seconds by casting it to string. This was replaced by special `TimestampFormatter` which appends the fraction formatter to `DateTimeFormatterBuilder`: `appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)`. The former one means trailing zeros in second's fraction should be truncated while formatting.
## How was this patch tested?
By existing test suites like `CastSuite`, `DateTimeUtilsSuite`, `JDBCSuite`, and by new test in `TimestampFormatterSuite`.
Closes#23936 from MaxGekk/timestamp-to-string.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
We have benchmark of nested schema pruning, but only for Parquet. This adds similar benchmark for ORC. This is used with nested schema pruning of ORC.
## How was this patch tested?
Added test.
Closes#23955 from viirya/orc-nested-schema-pruning-benchmark.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
This pr tried to drop `Serializable` in `WalkedTypePath`.
## How was this patch tested?
Pass Jenkins.
Closes#23973 from maropu/SPARK-27001-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/12881 removed `HiveSessionHook`. But there are still some code related to `HiveSessionHook`.
This PR removes all `HiveSessionHook` related code.
## How was this patch tested?
manual tests
Closes#23957 from wangyum/SPARK-15095.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This PR optimizes `InSet` expressions for byte, short, integer, date types. It is a follow-up on PR #21442 from dbtsai.
`In` expressions are compiled into a sequence of if-else statements, which results in O\(n\) time complexity. `InSet` is an optimized version of `In`, which is supposed to improve the performance if all values are literals and the number of elements is big enough. However, `InSet` actually worsens the performance in many cases due to various reasons.
The main idea of this PR is to use Java `switch` statements to significantly improve the performance of `InSet` expressions for bytes, shorts, ints, dates. All `switch` statements are compiled into `tableswitch` and `lookupswitch` bytecode instructions. We will have O\(1\) time complexity if our case values are compact and `tableswitch` can be used. Otherwise, `lookupswitch` will give us O\(log n\).
Locally, I tried Spark `OpenHashSet` and primitive collections from `fastutils` in order to solve the boxing issue in `InSet`. Both options significantly decreased the memory consumption and `fastutils` improved the time compared to `HashSet` from Scala. However, the switch-based approach was still more than two times faster even on 500+ non-compact elements.
I also noticed that applying the switch-based approach on less than 10 elements gives a relatively minor improvement compared to the if-else approach. Therefore, I placed the switch-based logic into `InSet` and added a new config to track when it is applied. Even if we migrate to primitive collections at some point, the switch logic will be still faster unless the number of elements is really big. Another option is to have a separate `InSwitch` expression. However, this would mean we need to modify other places (e.g., `DataSourceStrategy`).
See [here](https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-3.html#jvms-3.10) and [here](https://stackoverflow.com/questions/10287700/difference-between-jvms-lookupswitch-and-tableswitch) for more information.
This PR does not cover long values as Java `switch` statements cannot be used on them. However, we can have a follow-up PR with an approach similar to binary search.
## How was this patch tested?
There are new tests that verify the logic of the proposed optimization.
The performance was evaluated using existing benchmarks. This PR was also tested on an EC2 instance (OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 4.14.77-70.59.amzn1.x86_64, Intel(R) Xeon(R) CPU E5-2686 v4 2.30GHz).
## Notes
- [This link](http://hg.openjdk.java.net/jdk8/jdk8/langtools/file/30db5e0aaf83/src/share/classes/com/sun/tools/javac/jvm/Gen.java#l1153) contains source code that decides between `tableswitch` and `lookupswitch`. The logic was re-used in the benchmarks. See the `isLookupSwitch` method.
Closes#23171 from aokolnychyi/spark-26205.
Lead-authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8 as they use Hadoop's implementation underneath. I think these are all the places that this needs a mention in the user-facing docs.
## How was this patch tested?
Doc tests.
Closes#23962 from srowen/SPARK-26016.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When reading files with empty partition columns, we can avoid using JoinRow.
## How was this patch tested?
Existing unit tests.
Closes#23953 from gengliangwang/avoidJoinRow.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In master, `ElementAt` nullable is always true;
be1cadf16d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala (L1977)
But, If input is an array and foldable, we could make its nullability more precise.
This fix is based on SPARK-26637(#23566).
## How was this patch tested?
Added tests in `CollectionExpressionsSuite`.
Closes#23867 from maropu/SPARK-26965.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below operators such as Project, Aggregate, Window, Union etc. This is the initial piece of work that will be needed for
the subsequent work of moving the subquery rewrites to the beginning of optimization phase.
The larger PR is [here](https://github.com/apache/spark/pull/23211) . This PR addresses the comment at [link](https://github.com/apache/spark/pull/23211#issuecomment-445705922).
## How was this patch tested?
Added a new test suite LeftSemiAntiJoinPushDownSuite.
Closes#23750 from dilipbiswal/SPARK-19712-pushleftsemi.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Similar to `SaveMode`, we should remove streaming `OutputMode` from data source v2 API, and use operations that has clear semantic.
The changes are:
1. append mode: create `StreamingWrite` directly. By default, the `WriteBuilder` will create `Write` to append data.
2. complete mode: call `SupportsTruncate#truncate`. Complete mode means truncating all the old data and appending new data of the current epoch. `SupportsTruncate` has exactly the same semantic.
3. update mode: fail. The current streaming framework can't propagate the update keys, so v2 sinks are not able to implement update mode. In the future we can introduce a `SupportsUpdate` trait.
The behavior changes:
1. all the v2 sinks(foreach, console, memory, kafka, noop) don't support update mode. The fact is, previously all the v2 sinks implement the update mode wrong. None of them can really support it.
2. kafka sink doesn't support complete mode. The fact is, the kafka sink can only append data.
## How was this patch tested?
existing tests
Closes#23859 from cloud-fan/update.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR introduces leveraging of subquery filters for partition pruning in file source.
Subquery expressions are not allowed to be used for partition pruning in `FileSourceStrategy` now, instead a `FilterExec` is added around the `FileSourceScanExec` to do the job.
This PR optimizes the process by allowing partition pruning subquery expressions as partition filters.
## How was this patch tested?
Added new UT and run existing UTs especially SPARK-25482 and SPARK-24085 related ones.
Closes#23802 from peter-toth/SPARK-26893.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Reduce work in HDFSMetadataLogSuite test to possibly de-flake it.
## How was this patch tested?
Existing tests
Closes#23937 from srowen/SPARK-27032.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This patch proposes refactoring `serializerFor` method between `ScalaReflection` and `JavaTypeInference`, being consistent with what we refactored for `deserializerFor` in #23854.
This patch also extracts the logic on recording walk type path since the logic is duplicated across `serializerFor` and `deserializerFor` with `ScalaReflection` and `JavaTypeInference`.
## How was this patch tested?
Existing tests.
Closes#23908 from HeartSaVioR/SPARK-27001.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Use the maven plugin option `treatWarningsAsErrors` to make sure the warnings are treated as errors while generating the parser file. In the absence of it, we may inadvertently introducing problems while making grammar changes. Please refer to [PR-23897](https://github.com/apache/spark/pull/23897) to know more about the context.
## How was this patch tested?
We can use two ways to build Spark 1) sbt 2) Maven
This PR, we made a change to configure the maven antlr plugin to include a parameter that makes antlr4 report error on warning. However, when spark is built using sbt, we use the sbt antlr plugin which does not allow us to pass this additional compilation flag. More info on sbt-antlr plugin can be found at [link](https://github.com/ihji/sbt-antlr4/blob/master/src/main/scala/com/simplytyped/Antlr4Plugin.scala)
In summary, this fix only applicable when we use maven to build.
Closes#23925 from dilipbiswal/antlr_fix.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Currently we can use `df.printSchema` to discover the schema information for a query. We should have a way to describe the output schema of a query using SQL interface.
Example:
DESCRIBE SELECT * FROM desc_table
DESCRIBE QUERY SELECT * FROM desc_table
```SQL
spark-sql> create table desc_table (c1 int comment 'c1-comment', c2 decimal comment 'c2-comment', c3 string);
spark-sql> desc select * from desc_table;
c1 int c1-comment
c2 decimal(10,0) c2-comment
c3 string NULL
```
## How was this patch tested?
Added a new test under SQLQueryTestSuite and SparkSqlParserSuite
Closes#23883 from dilipbiswal/dkb_describe_query.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
I see the following new warning from ANTR4 after SPARK-26215 after it added `SCHEMA` keyword in the reserved/unreserved list. This is a minor PR to cleanup the warning.
```
WARNING] warning(125): org/apache/spark/sql/catalyst/parser/SqlBase.g4:784:90: implicit definition of token SCHEMA in parser
[WARNING] .../apache/spark/org/apache/spark/sql/catalyst/parser/SqlBase.g4 [784:90]: implicit definition of token SCHEMA in parser
```
## How was this patch tested?
Manually built catalyst after the fix to verify
Closes#23897 from dilipbiswal/minor_parser_token.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
Redundant `get` when getting a value from `Map` given a key.
## How was this patch tested?
N/A
Closes#23901 from 10110346/removegetfrommap.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to add new Catalyst type converter for `DateType`. It should be able to convert `java.time.LocalDate` to/from `DateType`.
Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Date` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.
By default, Spark converts values of `DateType` to `java.sql.Date` instances but the SQL config `spark.sql.datetime.java8API.enabled` can change the behavior. If it is set to `true`, Spark uses `java.time.LocalDate` as external type for `DateType`.
## How was this patch tested?
Added new testes to `CatalystTypeConvertersSuite` to check conversion of `DateType` to/from `java.time.LocalDate`, `JavaUDFSuite`/ `UDFSuite` to test usage of `LocalDate` type in Scala/Java UDFs.
Closes#23913 from MaxGekk/date-localdate.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Continuous processing is waiting on epochs which are not yet complete (for example one partition is not making progress) and stores pending items in queues. These queues are unbounded and can consume up all the memory easily. In this PR I've added `spark.sql.streaming.continuous.epochBacklogQueueSize` configuration possibility to make them bounded. If the related threshold reached then the query will stop with `IllegalStateException`.
## How was this patch tested?
Existing + additional unit tests.
Closes#23156 from gaborgsomogyi/SPARK-24063.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)
## How was this patch tested?
N/A
Closes#23884 from 10110346/simplifyboolean.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to add new Catalyst type converter for `TimestampType`. It should be able to convert `java.time.Instant` to/from `TimestampType`.
Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Timestamp` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.
By default, Spark converts values of `TimestampType` to `java.sql.Timestamp` instances but the SQL config `spark.sql.catalyst.timestampType` can change the behavior. It accepts two values `Timestamp` (default) and `Instant`. If the former one is set, Spark returns `java.time.Instant` instances for timestamp values.
## How was this patch tested?
Added new testes to `CatalystTypeConvertersSuite` to check conversion of `TimestampType` to/from `java.time.Instant`.
Closes#23811 from MaxGekk/timestamp-instant.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
WIth the following file structure:
```
/tmp/data
└── a=5
```
In the previous release:
```
scala> spark.read.schema("A int, ID long").parquet("/tmp/data/").printSchema
root
|-- ID: long (nullable = true)
|-- A: integer (nullable = true)
```
While in current code:
```
scala> spark.read.schema("A int, ID long").parquet("/tmp/data/").printSchema
root
|-- ID: long (nullable = true)
|-- a: integer (nullable = true)
```
We can see that the partition column name `a` is different from `A` as user specifed. This PR is to fix the case and make it more user-friendly.
## How was this patch tested?
Unit test
Closes#23894 from gengliangwang/fileIndexSchema.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Spark expects the type of column and the type of matching field is same when deserializing to Object, but Spark hasn't actually restrict it (at least for Java bean encoder) and some users just do it and experience undefined behavior (in SPARK-22000, Spark throws compilation failure on generated code because it calls `.toString()` against primitive type.
It doesn't produce error in Scala side because `ScalaReflection.deserializerFor` properly inject Upcast if necessary. This patch proposes applying same thing to `JavaTypeInference.deserializerFor` as well.
Credit to srowen, maropu, and cloud-fan since they provided various approaches to solve this.
## How was this patch tested?
Added UT which query is slightly modified based on sample code in attachment on JIRA issue.
Closes#23854 from HeartSaVioR/SPARK-22000.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR targets to add vectorized `dapply()` in R, Arrow optimization.
This can be tested as below:
```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
```r
df <- createDataFrame(mtcars)
collect(dapply(df, function(rdf) { data.frame(rdf$gear + 1) }, structType("gear double")))
```
### Requirements
- R 3.5.x
- Arrow package 0.12+
```bash
Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
```
**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.
### Benchmarks
**Shall**
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false --driver-memory 4g
```
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true --driver-memory 4g
```
**R code**
```r
rdf <- read.csv("500000.csv")
df <- cache(createDataFrame(rdf))
count(df)
test <- function() {
options(digits.secs = 6) # milliseconds
start.time <- Sys.time()
count(cache(dapply(df, function(rdf) { rdf }, schema(df))))
end.time <- Sys.time()
time.taken <- end.time - start.time
print(time.taken)
}
test()
```
**Data (350 MB):**
```r
object.size(read.csv("500000.csv"))
350379504 bytes
```
"500000 Records" http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/
**Results**
```
Time difference of 13.42037 mins
```
```
Time difference of 30.64156 secs
```
The performance improvement was around **2627%**.
### Limitations
- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.
- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.
## How was this patch tested?
Unit tests were added, and manually tested.
Closes#23787 from HyukjinKwon/SPARK-26830-1.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
In SPARK-26619, we make change to prune unnecessary individual serializers when serializing objects. This is extension to SPARK-26619. We can further prune nested fields from object serializers if they are not used.
For example, in following query, we only use one field in a struct column:
```scala
val data = Seq((("a", 1), 1), (("b", 2), 2), (("c", 3), 3))
val df = data.toDS().map(t => (t._1, t._2 + 1)).select("_1._1")
```
So, instead of having a serializer to create a two fields struct, we can prune unnecessary field from it. This is what this PR proposes to do.
In order to make this change conservative and safer, a SQL config is added to control it. It is disabled by default.
TODO: Support to prune nested fields inside MapType's key and value.
## How was this patch tested?
Added tests.
Closes#23740 from viirya/nested-pruning-serializer-2.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.
## How was this patch tested?
The changes are tested by existing test suites.
Closes#23878 from MaxGekk/magic-time-constants.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In data source v2, if the data source scan implemented `SupportsReportStatistics`. `DataSourceV2Relation` should respect the row count reported by the data source.
## How was this patch tested?
New UT test.
Closes#23853 from ConeyLiu/report-row-count.
Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a followup PR to resolve comment: https://github.com/apache/spark/pull/23601#pullrequestreview-207101115
When Spark writes DataFrame with "overwrite" mode, it deletes the output path before actual writes. To safely handle the case that the output path doesn't exist, it is suggested to follow the V1 code by checking the existence.
## How was this patch tested?
Apply https://github.com/apache/spark/pull/23836 and run unit tests
Closes#23889 from gengliangwang/checkFileBeforeOverwrite.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
Prior to this patch, all DataFrameReader.csv() calls would collect the first
line from the CSV input iterator. This is done to allow schema inference from the
header row.
However when schema is already specified this is a wasteful operation. It results
in an unncessary compute step on the first partition. This can be expensive if
the CSV itself is expensive to generate (e.g. it's the product of a long-running
external pipe()).
This patch short-circuits the first-line collection in DataFrameReader.csv() when
schema is specified. Thereby improving CSV read performance in certain cases.
## What changes were proposed in this pull request?
Short-circuiting DataFrameReader.csv() first-line read when schema is user-specified.
## How was this patch tested?
Compiled and tested against several CSV datasets.
Closes#23830 from Mister-Meeseeks/master.
Authored-by: Douglas R Colkitt <douglas.colkitt@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
The `DateTimeUtils.toMillis` can produce inaccurate result for some negative values (timestamps before epoch). The error can be around 1ms. In the PR, I propose to use `Math.floorDiv` in casting microseconds to milliseconds, and milliseconds to days since epoch.
## How was this patch tested?
Added new test to `DateTimeUtilsSuite`, and tested by `CastSuite` as well.
Closes#23815 from MaxGekk/micros-to-millis.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to convert time zone string to `TimeZone` by converting it to `ZoneId` which uses `ZoneOffset` internally. The `ZoneOffset` class of JDK 8 has a cache already: http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/687fd7c7986d/src/share/classes/java/time/ZoneOffset.java#l205 . In this way, there is no need to support cache of time zones in Spark.
The PR removes `computedTimeZones` from `DateTimeUtils`, and uses `ZoneId.of` to convert time zone id string to `ZoneId` and to `TimeZone` at the end.
## How was this patch tested?
The changes were tested by
Closes#23812 from MaxGekk/timezone-cache.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This pr targeted to define reserved/non-reserved keywords for Spark SQL based on the ANSI SQL standards and the other database-like systems (e.g., PostgreSQL). We assume that they basically follow the ANSI SQL-2011 standard, but it is slightly different between each other. Therefore, this pr documented all the keywords in `docs/sql-reserved-and-non-reserved-key-words.md`.
NOTE: This pr only added a small set of keywords as reserved ones and these keywords are reserved in all the ANSI SQL standards (SQL-92, SQL-99, SQL-2003, SQL-2008, SQL-2011, and SQL-2016) and PostgreSQL. This is because there is room to discuss which keyword should be reserved or not, .e.g., interval units (day, hour, minute, second, ...) are reserved in the ANSI SQL standards though, they are not reserved in PostgreSQL. Therefore, we need more researches about the other database-like systems (e.g., Oracle Databases, DB2, SQL server) in follow-up activities.
References:
- The reserved/non-reserved SQL keywords in the ANSI SQL standards: https://developer.mimer.com/wp-content/uploads/2018/05/Standard-SQL-Reserved-Words-Summary.pdf
- SQL Key Words in PostgreSQL: https://www.postgresql.org/docs/current/sql-keywords-appendix.html
## How was this patch tested?
Added tests in `TableIdentifierParserSuite`.
Closes#23259 from maropu/SPARK-26215-WIP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
Per https://github.com/apache/spark/pull/23768/files#r259083019 the last change to this line here caused the 2.11 build to fail. It's worked around by making `_cachedColumnBuffers` a field, as it was never set by callers to anything other than its default of null.
## How was this patch tested?
Existing tests.
Closes#23864 from srowen/SPARK-26851.2.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
Add assert to verify predicate class in ParquetFilterSuite
## How was this patch tested?
Ran ParquetFilterSuite, tests passed
Closes#23855 from nandorKollar/SPARK-26930.
Lead-authored-by: nandorKollar <nandorKollar@users.noreply.github.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: Nandor Kollar <nkollar@cloudera.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Apache Spark uses the predefined `Float.NaN` and `Double.NaN` for NaN values, but there exists more NaN values with different binary presentations.
```scala
scala> java.nio.ByteBuffer.allocate(4).putFloat(Float.NaN).array
res1: Array[Byte] = Array(127, -64, 0, 0)
scala> val x = java.lang.Float.intBitsToFloat(-6966608)
x: Float = NaN
scala> java.nio.ByteBuffer.allocate(4).putFloat(x).array
res2: Array[Byte] = Array(-1, -107, -78, -80)
```
Since users can have these values, `RandomDataGenerator` generates these NaN values. However, this causes `checkEvaluationWithUnsafeProjection` failures due to the difference between `UnsafeRow` binary presentation. The following is the UT failure instance. This PR aims to fix this UT flakiness.
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102528/testReport/
## How was this patch tested?
Pass the Jenkins with the newly added test cases.
Closes#23851 from dongjoon-hyun/SPARK-26950.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The function of the method named verifyTableProperties is
`If the given table properties contains datasource properties, throw an exception. We will do this check when create or alter a table, i.e. when we try to write table metadata to Hive metastore.`
But the message of AnalysisException in verifyTableProperties contains one typo and one unsuited word.
So I change the exception from
`Cannot persistent ${table.qualifiedName} into hive metastore`
to
`Cannot persist ${table.qualifiedName} into Hive metastore`
## How was this patch tested?
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23574 from beliefer/incorrect-analysis-exception.
Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This adds `NestedSchemaPruningBenchmark` to show the nested schema pruning performance clearly and to verify new PR's performance benefit and to prevent the future performance degradation.
## How was this patch tested?
Manually run the benchmark.
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.NestedSchemaPruningBenchmark"
```
Closes#23862 from dongjoon-hyun/SPARK-NESTED-SCHEMA-PRUNING-BM.
Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
Related to SPARK-26617 and SPARK-26548. There was a new location we found where we were still seeing the locks. We traced it to the recacheByCondition function. In this PR I have changed that function so that the writeLock is not held while the condition is being evaluated.
cloud-fan & gatorsmile This is a further tweak to the other cache PRs we have done (which have helped us tremendously).
## How was this patch tested?
Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.
Closes#23833 from DaveDeCaprio/cache-lock-recacheByCondition.
Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
When a user specifies a checkpoint location or a file sink output using a path containing special chars that need to be escaped in a path, the streaming query will store checkpoint and file sink metadata in a wrong place. In this PR, I uploaded a checkpoint that was generated by the following codes using Spark 2.4.0 to show this issue:
```
implicit val s = spark.sqlContext
val input = org.apache.spark.sql.execution.streaming.MemoryStream[Int]
input.addData(1, 2, 3)
val q = input.toDF.writeStream.format("parquet").option("checkpointLocation", ".../chk %#chk").start(".../output %#output")
q.stop()
```
Here is the structure of the directory:
```
sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0
├── chk%252520%252525%252523chk
│ ├── commits
│ │ └── 0
│ ├── metadata
│ └── offsets
│ └── 0
├── output %#output
│ └── part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet
└── output%20%25%23output
└── _spark_metadata
└── 0
```
In this checkpoint, the user specified checkpoint location is `.../chk %#chk` but the real path to store the checkpoint is `.../chk%252520%252525%252523chk` (this is generated by escaping the original path three times). The user specified output path is `.../output %#output` but the path to store `_spark_metadata` is `.../output%20%25%23output/_spark_metadata` (this is generated by escaping the original path once). The data files are still in the correct path (such as `.../output %#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet`).
This checkpoint will be used in unit tests in this PR.
The fix is just simply removing improper `Path.toUri` calls to fix the issue.
However, as the user may not read the release note and is not aware of this checkpoint location change, if they upgrade Spark without moving checkpoint to the new location, their query will just start from the scratch. In order to not surprise the users, this PR also adds a check to **detect the impacted paths and throws an error** to include the migration guide. This check can be turned off by an internal sql conf `spark.sql.streaming.checkpoint.escapedPathCheck.enabled`. Here are examples of errors that will be reported:
- Streaming checkpoint error:
```
Error: we detected a possible problem with the location of your checkpoint and you
likely need to move it before restarting this query.
Earlier version of Spark incorrectly escaped paths when writing out checkpoints for
structured streaming. While this was corrected in Spark 3.0, it appears that your
query was started using an earlier version that incorrectly handled the checkpoint
path.
Correct Checkpoint Directory: /.../chk %#chk
Incorrect Checkpoint Directory: /.../chk%252520%252525%252523chk
Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```
- File sink error (`_spark_metadata`):
```
Error: we detected a possible problem with the location of your "_spark_metadata"
directory and you likely need to move it before restarting this query.
Earlier version of Spark incorrectly escaped paths when writing out the
"_spark_metadata" directory for structured streaming. While this was corrected in
Spark 3.0, it appears that your query was started using an earlier version that
incorrectly handled the "_spark_metadata" path.
Correct "_spark_metadata" Directory: /.../output %#output/_spark_metadata
Incorrect "_spark_metadata" Directory: /.../output%20%25%23output/_spark_metadata
Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```
## How was this patch tested?
The new unit tests.
Closes#23733 from zsxwing/path-fix.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
In the PR, I propose to simplify timestamp truncation to quarter of year by using *java.time* API directly. The `LocalDate` instance can be truncation to quarter timestamp via adjusting by chrono field `IsoFields.DAY_OF_QUARTER`.
## How was this patch tested?
This was checked by existing test suite - `DateTimeUtilsSuite`.
Closes#23808 from MaxGekk/date-quarter-of-year.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This happens in a schema evolution use case only when a user specifies the schema manually and use non-vectorized ORC deserializer code path.
There is a bug in `OrcDeserializer.scala` that results in `null`s being set at the wrong column position, and for state from previous records to remain uncleared in next records. There are more details for when exactly the bug gets triggered and what the outcome is in the [JIRA issue](https://jira.apache.org/jira/browse/SPARK-26859).
The high-level summary is that this bug results in severe data correctness issues, but fortunately the set of conditions to expose the bug are complicated and make the surface area somewhat small.
This change fixes the problem and adds a respective test.
## How was this patch tested?
Pass the Jenkins with the newly added test cases.
Closes#23766 from IvanVergiliev/fix-orc-deserializer.
Lead-authored-by: Ivan Vergiliev <ivan.vergiliev@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR targets to support Arrow optimization for conversion from Spark DataFrame to R DataFrame.
Like PySpark side, it falls back to non-optimization code path when it's unable to use Arrow optimization.
This can be tested as below:
```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
```r
collect(createDataFrame(mtcars))
```
### Requirements
- R 3.5.x
- Arrow package 0.12+
```bash
Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
```
**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.
### Benchmarks
**Shall**
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false --driver-memory 4g
```
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true --driver-memory 4g
```
**R code**
```r
df <- cache(createDataFrame(read.csv("500000.csv")))
count(df)
test <- function() {
options(digits.secs = 6) # milliseconds
start.time <- Sys.time()
collect(df)
end.time <- Sys.time()
time.taken <- end.time - start.time
print(time.taken)
}
test()
```
**Data (350 MB):**
```r
object.size(read.csv("500000.csv"))
350379504 bytes
```
"500000 Records" http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/
**Results**
```
Time difference of 221.32014 secs
```
```
Time difference of 15.51145 secs
```
The performance improvement was around **1426%**.
### Limitations:
- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values. In this case, we decide to fall back to non-optimization code path.
- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.
## How was this patch tested?
Existing tests related with Arrow optimization cover this change. Also, manually tested.
Closes#23760 from HyukjinKwon/SPARK-26762.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is a followup PR for #21149.
New way uses unsafeRow.hashCode() as hash value in HashAggregate.
The unsafe row has [null bit set] etc., so the hash should be different from shuffle hash, and then we don't need a special seed.
## How was this patch tested?
UTs.
Closes#23821 from yucai/unsafe_hash.
Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This proposes adding a note on `QueryExecution.toRdd` regarding Spark's internal optimization callers would need to indicate.
## How was this patch tested?
This patch is a documentation change.
Closes#23822 from HeartSaVioR/MINOR-doc-add-note-query-execution-to-rdd.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Continue the API refactor for streaming write, according to the [doc](https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing).
The major changes:
1. rename `StreamingWriteSupport` to `StreamingWrite`
2. add `WriteBuilder.buildForStreaming`
3. update existing sinks, to move the creation of `StreamingWrite` to `Table`
## How was this patch tested?
existing tests
Closes#23702 from cloud-fan/stream-write.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR partially revert SPARK-26744.
60caa92dea and 4dce45a599 were merged at similar time range independently. So the test failures were not caught.
- 60caa92dea happened to add a schema reading logic in writing path for overwrite mode as well.
- 4dce45a599 added some tests with overwrite modes with migrated ORC v2.
And the tests looks starting to fail.
I guess the discussion won't be short (see https://github.com/apache/spark/pull/23606#discussion_r257675083) and this PR proposes to disable the tests added at 4dce45a599 to unblock other PRs for now.
## How was this patch tested?
Existing tests.
Closes#23828 from HyukjinKwon/SPARK-26744.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This adds two logical plans that implement the ReplaceData operation from the [logical plans SPIP](https://docs.google.com/document/d/1gYm5Ji2Mge3QBdOliFV5gSPTKlX4q1DCBXIkiyMv62A/edit?ts=5a987801#heading=h.m45webtwxf2d). These two plans will be used to implement Spark's `INSERT OVERWRITE` behavior for v2.
Specific changes:
* Add `SupportsTruncate`, `SupportsOverwrite`, and `SupportsDynamicOverwrite` to DSv2 write API
* Add `OverwriteByExpression` and `OverwritePartitionsDynamic` plans (logical and physical)
* Add new plans to DSv2 write validation rule `ResolveOutputRelation`
* Refactor `WriteToDataSourceV2Exec` into trait used by all DSv2 write exec nodes
## How was this patch tested?
* The v2 analysis suite has been updated to validate the new overwrite plans
* The analysis suite for `OverwriteByExpression` checks that the delete expression is resolved using the table's columns
* Existing tests validate that overwrite exec plan works
* Updated existing v2 test because schema is used to validate overwrite
Closes#23606 from rdblue/SPARK-26666-add-overwrite.
Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This pr just removed workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite.
## How was this patch tested?
Pass the Jenkins.
Closes#23817 from maropu/SPARK-26607-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
The previous strategy for comparing Maps leveraged sorting (key, value) tuples by their _.toString. However, the _.toString representation of an arrays has nothing to do with it's content. If a map has array keys, it's (key, value) pairs would be compared with other maps essentially at random. This could results in false negatives in tests.
This changes first compares keys together to find the matching ones, and then compares associated values.
## How was this patch tested?
New unit test added.
Closes#23789 from ala/compare-map.
Authored-by: Ala Luszczak <ala@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The maintenance release of `branch-2.3` (v2.3.3) vote passed, so this issue updates PROCESS_TABLES.testingVersions in HiveExternalCatalogVersionsSuite
## How was this patch tested?
Pass the Jenkins.
Closes#23807 from maropu/SPARK-26897.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
The file source has a schema validation feature, which validates 2 schemas:
1. the user-specified schema when reading.
2. the schema of input data when writing.
If a file source doesn't support the schema, we can fail the query earlier.
This PR is to implement the same feature in the `FileDataSourceV2` framework. Comparing to `FileFormat`, `FileDataSourceV2` has multiple layers. The API is added in two places:
1. Read path: the table schema is determined in `TableProvider.getTable`. The actual read schema can be a subset of the table schema. This PR proposes to validate the actual read schema in `FileScan`.
2. Write path: validate the actual output schema in `FileWriteBuilder`.
## How was this patch tested?
Unit test
Closes#23714 from gengliangwang/schemaValidationV2.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a followup PR to fix two issues in #23601:
1. the class `FileWriterFactory` contains `conf: SerializableConfiguration` as a member, which is duplicated with `WriteJobDescription. serializableHadoopConf `. By removing it we can reduce the broadcast task binary size by around 70KB
2. The test suite `OrcV1QuerySuite`/`OrcV1QuerySuite`/`OrcV1PartitionDiscoverySuite` didn't change the configuration `SQLConf.USE_V1_SOURCE_WRITER_LIST` to `"orc"`. We should set the conf.
## How was this patch tested?
Unit test
Closes#23800 from gengliangwang/reduceWriteTaskSize.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
`HadoopDelegationTokenProvider` has basically the same functionality just like `ServiceCredentialProvider` so the interfaces can be merged.
`YARNHadoopDelegationTokenManager` now loads `ServiceCredentialProvider`s in one step. The drawback of this if one provider fails all others are not loaded. `HadoopDelegationTokenManager` loads `HadoopDelegationTokenProvider`s independently so it provides more robust behaviour.
In this PR I've I've made the following changes:
* Deleted `YARNHadoopDelegationTokenManager` and `ServiceCredentialProvider`
* Made `HadoopDelegationTokenProvider` a `DeveloperApi`
## How was this patch tested?
Existing unit tests.
Closes#23686 from gaborgsomogyi/SPARK-26772.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/23383, the file source V2 framework is implemented. In the PR, `FileIndex` is created as a member of `FileTable`, so that we can implement partition pruning like 0f9fcabb4a in the future(As data source V2 catalog is under development, partition pruning is removed from the PR)
However, after write path of file source V2 is implemented, I find that a simple write will create an unnecessary `FileIndex`, which is required by `FileTable`. This is a sort of regression. And we can see there is a warning message when writing to ORC files
```
WARN InMemoryFileIndex: The directory file:/tmp/foo was not found. Was it deleted very recently?
```
This PR is to make `FileIndex` as a lazy value in `FileTable`, so that we can avoid creating unnecessary `FileIndex` in the write path.
## How was this patch tested?
Existing unit test
Closes#23774 from gengliangwang/moveFileIndexInV2.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a fix for https://github.com/apache/spark/pull/23524, which did not stop cost-based join reorder when the CostBasedJoinReorder rule recurses down the tree and applies join reorder for nested joins with hints.
The issue had not been detected by the existing tests because CBO is disabled by default.
## How was this patch tested?
Enabled CBO for JoinHintSuite.
Closes#23759 from maryannxue/spark-26840.
Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
These builtin typed aggregate functions are not very useful:
1. users can just call the untyped ones and turn the resulting dataframe to a dataset. It has better performance.
2. the typed aggregate functions have subtle different behaviors regarding empty input.
I think we should get rid of these builtin typed agg functions and suggest users to use the untyped ones.
However, these functions are still useful as a demo of the `Aggregator` API, so I copied them to the example module.
## How was this patch tested?
N/A
Closes#23763 from cloud-fan/example.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Updates FileFormatWriter to create a consistent Hadoop Job ID for a write.
## How was this patch tested?
Existing tests for regressions.
Closes#23777 from rdblue/SPARK-26873-fix-file-format-writer-job-ids.
Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
This PR is a correctness fix in `HashAggregateExec` code generation. It forces evaluation of result expressions before calling `consume()` to avoid multiple executions.
This PR fixes a use case where an aggregate is nested into a broadcast join and appears on the "stream" side. The issue is that Broadcast join generates it's own loop. And without forcing evaluation of `resultExpressions` of `HashAggregateExec` before the join's loop these expressions can be executed multiple times giving incorrect results.
## How was this patch tested?
New UT was added.
Closes#23731 from peter-toth/SPARK-26572.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
```java
public class SqlDemo {
public static void main(final String[] args) throws Exception {
SparkConf conf = new SparkConf().setAppName("spark-sql-demo");
JavaSparkContext sc = new JavaSparkContext(conf);
SparkSession ss = SparkSession.builder().enableHiveSupport().getOrCreate();
ss.sql("show databases").show();
}
}
```
Before https://issues.apache.org/jira/browse/SPARK-20946, the demo above point to the right hive metastore if the hive-site.xml is present. But now it can only point to the default in-memory one.
Catalog is now as a variable shared across SparkSessions, it is instantiated with SparkContext's conf. After https://issues.apache.org/jira/browse/SPARK-20946, Session level configs are not pass to SparkContext's conf anymore, so the enableHiveSupport API takes no affect on the catalog instance.
You can set spark.sql.catalogImplementation=hive application wide to solve the problem, or never create a sc before you call SparkSession.builder().enableHiveSupport().getOrCreate()
Here we respect the SparkSession level configuration at the first time to generate catalog within SharedState
## How was this patch tested?
1. add ut
2. manually
```scala
test("enableHiveSupport has right to determine the catalog while using an existing sc") {
val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
val sc = SparkContext.getOrCreate(conf)
val ss = SparkSession.builder().enableHiveSupport().getOrCreate()
assert(ss.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
"The catalog should be hive ")
val ss2 = SparkSession.builder().getOrCreate()
assert(ss2.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
"The catalog should be shared across sessions")
}
```
Without this fix, the above test will fail.
You can apply it to `org.apache.spark.sql.hive.HiveSharedStateSuite`,
and run,
```sbt
./build/sbt -Phadoop-2.7 -Phive "hive/testOnly org.apache.spark.sql.hive.HiveSharedStateSuite"
```
to verify.
Closes#23709 from yaooqinn/SPARK-26794.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
According to Brian Goetz et al in Java Concurrency in Practice, the double checked locking pattern has worked since Java 5, but only if the resource is declared volatile:
> Subsequent changes in the JMM (Java 5.0 and later) have enabled DCL to work if resource is made volatile, and the performance impact of this is small since volatile reads are usually only slightly more expensive than nonvolatile reads.
CachedRDDBuilder. cachedColumnBuffers and CachedRDDBuilder.clearCache both use DCL to manage the resource ``_cachedColumnBuffers``. The missing ingredient is that ``_cachedColumnBuffers`` is not volatile.
Because of this, clearCache may see ``_cachedColumnBuffers`` as null, when in fact it is not, and therefore fail to un-cache the RDD. There may be other, more subtle bugs due to visibility issues.
To avoid these issues, this PR makes ``_cachedColumnBuffers`` volatile.
## How was this patch tested?
- Existing SQL unit tests
- Existing pyspark-sql tests
Closes#23768 from bersprockets/SPARK-26851.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to use `System.nanoTime()` instead of `System.currentTimeMillis()` in measurements of time intervals.
`System.currentTimeMillis()` returns current wallclock time and will follow changes to the system clock. Thus, negative wallclock adjustments can cause timeouts to "hang" for a long time (until wallclock time has caught up to its previous value again). This can happen when ntpd does a "step" after the network has been disconnected for some time. The most canonical example is during system bootup when DHCP takes longer than usual. This can lead to failures that are really hard to understand/reproduce. `System.nanoTime()` is guaranteed to be monotonically increasing irrespective of wallclock changes.
## How was this patch tested?
By existing test suites.
Closes#23727 from MaxGekk/system-nanotime.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
There is a very old TODO in `HandleNullInputsForUDF`, saying that we can skip the null check if input is not nullable. We leverage the nullability info at many places, we can trust it here too.
## How was this patch tested?
re-enable an ignored test
Closes#23712 from cloud-fan/minor.
Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
In SPARK-25314, we supported the scenario of having a python UDF that refers to attributes from both legs of a join condition by rewriting the plan to convert an inner join or left semi join to a filter over a cross join. In case of left semi join, this transformation may cause incorrect results when the right leg of join condition produces duplicate rows based on the join condition. This fix disallows the rewrite for left semi join and raises an error in the case like we do for other types of join. In future, we should have separate rule in optimizer to convert left semi join to inner join (I am aware of one case we could do it if we leverage informational constraint i.e when we know the right side does not produce duplicates).
**Python**
```SQL
>>> from pyspark import SparkContext
>>> from pyspark.sql import SparkSession, Column, Row
>>> from pyspark.sql.functions import UserDefinedFunction, udf
>>> from pyspark.sql.types import *
>>> from pyspark.sql.utils import AnalysisException
>>>
>>> spark.conf.set("spark.sql.crossJoin.enabled", "True")
>>> left = spark.createDataFrame([Row(lc1=1, lc2=1), Row(lc1=2, lc2=2)])
>>> right = spark.createDataFrame([Row(rc1=1, rc2=1), Row(rc1=1, rc2=1)])
>>> func = udf(lambda a, b: a == b, BooleanType())
>>> df = left.join(right, func("lc1", "rc1"), "leftsemi").show()
19/02/12 16:07:10 WARN PullOutPythonUDFInJoinCondition: The join condition:<lambda>(lc1#0L, rc1#4L) of the join plan contains PythonUDF only, it will be moved out and the join plan will be turned to cross join.
+---+---+
|lc1|lc2|
+---+---+
| 1| 1|
| 1| 1|
+---+---+
```
**Scala**
```SQL
scala> val left = Seq((1, 1), (2, 2)).toDF("lc1", "lc2")
left: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]
scala> val right = Seq((1, 1), (1, 1)).toDF("rc1", "rc2")
right: org.apache.spark.sql.DataFrame = [rc1: int, rc2: int]
scala> val equal = udf((p1: Integer, p2: Integer) => {
| p1 == p2
| })
equal: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$2141/11016292394666f1b5,BooleanType,List(Some(Schema(IntegerType,true)), Some(Schema(IntegerType,true))),None,false,true)
scala> val df = left.join(right, equal(col("lc1"), col("rc1")), "leftsemi")
df: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]
scala> df.show()
+---+---+
|lc1|lc2|
+---+---+
| 1| 1|
+---+---+
```
## How was this patch tested?
Modified existing tests.
Closes#23769 from dilipbiswal/dkb_python_udf_in_join.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR targets to add vectorized `gapply()` in R, Arrow optimization.
This can be tested as below:
```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
```r
df <- createDataFrame(mtcars)
collect(gapply(df,
"gear",
function(key, group) {
data.frame(gear = key[[1]], disp = mean(group$disp) > group$disp)
},
structType("gear double, disp boolean")))
```
### Requirements
- R 3.5.x
- Arrow package 0.12+
```bash
Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
```
**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.
### Benchmarks
**Shall**
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false
```
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
**R code**
```r
rdf <- read.csv("500000.csv")
rdf <- rdf[, c("Month.of.Joining", "Weight.in.Kgs.")] # We're only interested in the key and values to calculate.
df <- cache(createDataFrame(rdf))
count(df)
test <- function() {
options(digits.secs = 6) # milliseconds
start.time <- Sys.time()
count(gapply(df,
"Month_of_Joining",
function(key, group) {
data.frame(Month_of_Joining = key[[1]], Weight_in_Kgs_ = mean(group$Weight_in_Kgs_) > group$Weight_in_Kgs_)
},
structType("Month_of_Joining integer, Weight_in_Kgs_ boolean")))
end.time <- Sys.time()
time.taken <- end.time - start.time
print(time.taken)
}
test()
```
**Data (350 MB):**
```r
object.size(read.csv("500000.csv"))
350379504 bytes
```
"500000 Records" http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/
**Results**
```
Time difference of 35.67459 secs
```
```
Time difference of 4.301399 secs
```
The performance improvement was around **829%**.
**Note that** I am 100% sure this PR improves more then 829% because I gave up testing it with non-Arrow optimization because it took super super super long when the data size becomes bigger.
### Limitations
- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.
- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.
## How was this patch tested?
Unit tests were added
**TODOs:**
- [x] Draft codes
- [x] make the tests passed
- [x] make the CRAN check pass
- [x] Performance measurement
- [x] Supportability investigation (for instance types)
Closes#23746 from HyukjinKwon/SPARK-26759.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
In https://github.com/apache/spark/issues/23569, the copy method of `ColumnarArray` is implemented.
To further improve it, we can return `UnsafeArrayData` for `date`/`timestamp` type in `ColumnarArray.copy()`.
## How was this patch tested?
Unit test
Closes#23761 from gengliangwang/copyDateAndTS.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
This improves the expression description for commonly used aggregate functions such as Max, Min, Count, etc.
## How was this patch tested?
Verified the function description manually from the shell.
Closes#23756 from dilipbiswal/dkb_expr_description_aggregate.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
Run Spark-Sql job use transform features(`ScriptTransformationExec`) with config `spark.speculation = true`, sometimes job fails and we found many Executor Dead through `Executor Tab`, through analysis log and code we found :
`ScriptTransformationExec` start a new thread(`ScriptTransformationWriterThread`), the new thread is very likely to throw `TaskKilledException`(from iter.map.foreach part) when speculation is on, this exception will captured by `SparkUncaughtExceptionHandler` which registered during Executor start, `SparkUncaughtExceptionHandler` will call `System.exit (SparkExitCode.UNCAUGHT_EXCEPTION)` to shutdown `Executor`, this is unexpected.
We should not kill the executor just because `ScriptTransformationWriterThread` fails. log the error(not only `TaskKilledException`) instead of throwing it is enough, Exception already pass to `ScriptTransformationExec` and handle by `TaskRunner`.
## How was this patch tested?
Register `TestUncaughtExceptionHandler` to test case in `ScriptTransformationSuite`, then assert there is no Uncaught Exception handled.
Before this patch "script transformation should not swallow errors from upstream operators (no serde)" and "script transformation should not swallow errors from upstream operators (with serde)" throwing `IllegalArgumentException` and handle by `TestUncaughtExceptionHandler` .
Closes#22149 from LuciferYang/fix-transformation-task-kill.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Implements the solution proposed in [SPARK-26696](https://issues.apache.org/jira/browse/SPARK-26696), a minor refactoring that allows frameworks to perform advanced type-preserving dataset transformations without carrying `Encoder` implicits from user code.
The change allows
```scala
def foo[A](ds: Dataset[A]): Dataset[A] =
ds.toDF().as[A](ds.encoder)
```
instead of
```scala
def foo[A: Encoder](ds: Dataset[A]): Dataset[A] =
ds.toDF().as[A](implicitly[Encoder[A]])
```
## How was this patch tested?
This patch was tested with an automated test that was later removed as it was deemed unnecessary per the discussion in this PR.
Closes#23620 from ssimeonov/ss_SPARK-26696.
Authored-by: Simeon Simeonov <sim@fastignite.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to covert underlying types of timestamp/date columns to strings, and store the converted values as column statistics. This makes statistics for timestamp/date columns independent from system time zone while saving and retrieving such statistics.
I bumped versions of stored statistics from 1 to 2 since the PR changes the format.
## How was this patch tested?
The changes were tested by `StatisticsCollectionSuite` and by `StatisticsSuite`.
Closes#23662 from MaxGekk/column-stats-time-date.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Delegation token providers interface now has a parameter `fileSystems` but this is needed only for `HadoopFSDelegationTokenProvider`.
In this PR I've addressed this issue in the following way:
* Removed `fileSystems` parameter from `HadoopDelegationTokenProvider`
* Moved `YarnSparkHadoopUtil.hadoopFSsToAccess` into `HadoopFSDelegationTokenProvider`
* Moved `spark.yarn.stagingDir` into core
* Moved `spark.yarn.access.namenodes` into core and renamed to `spark.kerberos.access.namenodes`
* Moved `spark.yarn.access.hadoopFileSystems` into core and renamed to `spark.kerberos.access.hadoopFileSystems`
## How was this patch tested?
Existing unit tests.
Closes#23698 from gaborgsomogyi/SPARK-26766.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Not all users wants to keep temporary checkpoint directories. Additionally hard to restore from it.
In this PR I've added a force delete flag which is default `false`. Additionally not clear for users when temporary checkpoint directory deleted so added log messages to explain this a bit more.
## How was this patch tested?
Existing + additional unit tests.
Closes#23732 from gaborgsomogyi/SPARK-26389.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
This PR consists of the `test` components of #23665 only, minus the associated patch from that PR.
It adds a new unit test to `JsonSuite` which verifies that the `count()` returned from a `DataFrame` loaded from JSON containing empty lines does not include those empty lines in the record count. The test runs `count` prior to otherwise reading data from the `DataFrame`, so as to catch future cases where a pre-parsing optimization might result in `count` results inconsistent with existing behavior.
This PR is intended to be deployed alongside #23667; `master` currently causes the test to fail, as described in [SPARK-26745](https://issues.apache.org/jira/browse/SPARK-26745).
## How was this patch tested?
Manual testing, existing `JsonSuite` unit tests.
Closes#23674 from sumitsu/json_emptyline_count_test.
Authored-by: Branden Smith <branden.smith@publicismedia.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When we run in background and we get exception which is not HiveSQLException,
we may encounter memory leak since handleToOperation will not removed correctly.
The reason is below:
1. When calling operation.run() in HiveSessionImpl#executeStatementInternal we throw an exception which is not HiveSQLException
2. Then the opHandle generated by SparkSQLOperationManager will not be added into opHandleSet of HiveSessionImpl , and operationManager.closeOperation(opHandle) will not be called
3. When we close the session we will also call operationManager.closeOperation(opHandle),since we did not add this opHandle into the opHandleSet.
For the reasons above,the opHandled will always in SparkSQLOperationManager#handleToOperation,which will cause memory leak.
More details and a case has attached on https://issues.apache.org/jira/browse/SPARK-26751
This patch will always throw HiveSQLException when running in background
## How was this patch tested?
Exist UT
Closes#23673 from caneGuy/zhoukang/fix-hivesessionimpl-leak.
Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to eliminate checking of parsed segments inside of the `stringToDate` and `stringToTimestamp` because such checking is already performed while constructing *java.time* classes, in particular inside of `LocalDate` and `LocalTime`. As a consequence of removing the explicit checks, the `isInvalidDate` method is not needed any more, and it was removed from `DateTimeUtils`.
## How was this patch tested?
This was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite` and `CastSuite`.
Closes#23717 from MaxGekk/datetimeutils-refactoring.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Update to Parquet Java 1.10.1.
## How was this patch tested?
Added a test from HyukjinKwon that validates the notEq case from SPARK-26677.
Closes#23704 from rdblue/SPARK-26677-fix-noteq-parquet-bug.
Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Ryan Blue <rdblue@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.
This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.
## How was this patch tested?
Existing tests.
Closes#23685 from srowen/SPARK-26771.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` correctly. This will make `avg` become `NaN`. And whatever gets merged with the result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call `NaN.toLong` and get `0`, and the user will see the following incorrect report:
```
"eventTime" : {
"avg" : "1970-01-01T00:00:00.000Z",
"max" : "2019-01-31T12:57:00.000Z",
"min" : "2019-01-30T18:44:04.000Z",
"watermark" : "1970-01-01T00:00:00.000Z"
}
```
This issue was reported by liancheng .
This PR fixes the above issue.
## How was this patch tested?
The new unit tests.
Closes#23718 from zsxwing/merge-zero.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
For types like Product, we've already add AssertNotNull when we construct serializer(see code below), so we could strip redundant AssertNotNull for those types.
```
val fieldValue = Invoke(
AssertNotNull(inputObject, walkedTypePath), fieldName, dataTypeFor(fieldType),
returnNullable = !fieldType.typeSymbol.asClass.isPrimitive)
```
## How was this patch tested?
Existed.
Closes#23651 from Ngone51/dev-strip-redundant-assertnotnull-for-ecnoder-serializer.
Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Create a framework for write path of File Source V2.
Also, migrate write path of ORC to V2.
Supported:
* Write to file as Dataframe
Not Supported:
* Partitioning, which is still under development in the data source V2 project.
* Bucketing, which is still under development in the data source V2 project.
* Catalog.
## How was this patch tested?
Unit test
Closes#23601 from gengliangwang/orc_write.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
A followup of https://github.com/apache/spark/pull/23388 .
`AggUtils.createAggregate` is not the right place to normalize the grouping expressions, as final aggregate is also created by it. The grouping expressions of final aggregate should be attributes which refer to the grouping expressions in partial aggregate.
This PR moves the normalization to the caller side of `AggUtils`.
## How was this patch tested?
existing tests
Closes#23692 from cloud-fan/follow.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In #23639, the API `supportDataType` is refactored. We should also remove the method `verifyWriteSchema` and `verifyReadSchema` in `DataSourceUtils`.
Since the error message use `FileFormat.toString` to specify the data source naming, this PR also overriding the `toString` method in `AvroFileFormat`.
## How was this patch tested?
Unit test.
Closes#23699 from gengliangwang/SPARK-26716-followup.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR reverts JSON count optimization part of #21909.
We cannot distinguish the cases below without parsing:
```
[{...}, {...}]
```
```
[]
```
```
{...}
```
```bash
# empty string
```
when we `count()`. One line (input: IN) can be, 0 record, 1 record and multiple records and this is dependent on each input.
See also https://github.com/apache/spark/pull/23665#discussion_r251276720.
## How was this patch tested?
Manually tested.
Closes#23667 from HyukjinKwon/revert-SPARK-24959.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Hive 3.1.1 is released. This PR aims to support Hive 3.1.x metastore.
Please note that Hive 3.0.0 Metastore is skipped intentionally.
## How was this patch tested?
Pass the Jenkins with the updated test cases including 3.1.
Closes#23694 from dongjoon-hyun/SPARK-24360-3.1.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This is a follow up of https://github.com/apache/spark/pull/23644/files , to make these methods less coupled with each other.
## How was this patch tested?
existing tests
Closes#23687 from cloud-fan/cache.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
After [recent changes](11e5f1bcd4) to CSV parsing to return partial results for bad CSV records, queries of wide CSV tables slowed considerably. That recent change resulted in every row being recreated, even when the associated input record had no parsing issues and the user specified no corrupt record field in his/her schema.
The change to FailureSafeParser.scala also impacted queries against wide JSON tables as well.
In this PR, I propose that a row should be recreated only if columns need to be shifted due to the existence of a corrupt column field in the user-supplied schema. Otherwise, the code should use the row as-is (For CSV input, it will have values for the columns that could be converted, and also null values for columns that could not be converted).
See benchmarks below. The CSV benchmark for 1000 columns went from 120144 ms to 89069 ms, a savings of 25% (this only brings the cost down to baseline levels. Again, see benchmarks below).
Similarly, the JSON benchmark for 1000 columns (added in this PR) went from 109621 ms to 80871 ms, also a savings of 25%.
Still, partial results functionality is preserved:
<pre>
bash-3.2$ cat test2.csv
"hello",1999-08-01,"last"
"there","bad date","field"
"again","2017-11-22","in file"
bash-3.2$ bin/spark-shell
...etc...
scala> val df = spark.read.schema("a string, b date, c string").csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [a: string, b: date ... 1 more field]
scala> df.show
+-----+----------+-------+
| a| b| c|
+-----+----------+-------+
|hello|1999-08-01| last|
|there| null| field|
|again|2017-11-22|in file|
+-----+----------+-------+
scala> val df = spark.read.schema("badRecord string, a string, b date, c string").
| option("columnNameOfCorruptRecord", "badRecord").
| csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [badRecord: string, a: string ... 2 more fields]
scala> df.show
+--------------------+-----+----------+-------+
| badRecord| a| b| c|
+--------------------+-----+----------+-------+
| null|hello|1999-08-01| last|
|"there","bad date...|there| null| field|
| null|again|2017-11-22|in file|
+--------------------+-----+----------+-------+
scala>
</pre>
### CSVBenchmark Benchmarks:
baseline = commit before partial results change
PR = this PR
master = master branch
[baseline_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697109/baseline_CSVBenchmark-results.txt)
[pr_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697110/pr_CSVBenchmark-results.txt)
[master_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697111/master_CSVBenchmark-results.txt)
### JSONBenchmark Benchmarks:
baseline = commit before partial results change
PR = this PR
master = master branch
[baseline_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711040/baseline_JSONBenchmark-results.txt)
[pr_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711041/pr_JSONBenchmark-results.txt)
[master_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711042/master_JSONBenchmark-results.txt)
## How was this patch tested?
- All SQL unit tests.
- Added 2 CSV benchmarks
- Python core and SQL tests
Closes#23336 from bersprockets/csv-wide-row-opt2.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
For making test suite supporting both Parquet and Orc by reusing test cases, this patch extracts the methods for testing. For example, if we need to test a common feature shared by Parquet and Orc, we should be able to write test cases once and reuse them to test both formats.
This patch extracts the methods for testing and uses a variable `dataSourceName` to set up data format to test against with.
## How was this patch tested?
Existing tests.
Closes#23628 from viirya/datasource-test.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
We should pass the existed `fileStatusCache` to `InMemoryFileIndex` even though there aren't partition columns.
## How was this patch tested?
Existed test. Extra tests can be added if there is a requirement.
Closes#23683 from ConeyLiu/filestatuscache.
Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Following https://github.com/apache/spark/pull/23430, this PR does the API refactor for continuous read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)
The major changes:
1. rename `XXXContinuousReadSupport` to `XXXContinuousStream`
2. at the beginning of continuous streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.
3. remove all the hacks as we have finished all the read side API refactor
## How was this patch tested?
existing tests
Closes#23619 from cloud-fan/continuous.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
In master, `GetMapValue` nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L371)
But, If input key is foldable, we could make its nullability more precise.
This fix is the same with SPARK-26637(#23566).
## How was this patch tested?
Added tests in `ComplexTypeSuite`.
Closes#23669 from maropu/SPARK-26747.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
- Replacing `java.util.Calendar` in `DateTimeUtils. truncTimestamp` and in `DateTimeUtils.getOffsetFromLocalMillis ` by equivalent code using Java 8 API for timestamp manipulations. The reason is `java.util.Calendar` is based on the hybrid calendar (Julian+Gregorian) but *java.time* classes use Proleptic Gregorian calendar which assumes by SQL standard.
- Replacing `Calendar.getInstance()` in `DateTimeUtilsSuite` by similar code in `DateTimeTestUtils` using *java.time* classes
## How was this patch tested?
The changes were tested by existing suites: `DateExpressionsSuite`, `DateFunctionsSuite` and `DateTimeUtilsSuite`.
Closes#23641 from MaxGekk/cleanup-date-time-utils.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Added the following benchmarks:
- Extract components from timestamp like year, month, day and etc.
- Current date and time
- Date arithmetic like date_add, date_sub
- Format dates and timestamps
- Convert timestamps from/to UTC
Closes#23661 from MaxGekk/datetime-benchmark.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
Adjust mem settings in UnifiedMemoryManager used in test suites to ha…ve execution memory > 0
Ref: https://github.com/apache/spark/pull/23457#issuecomment-457409976
## How was this patch tested?
Existing tests
Closes#23645 from srowen/SPARK-26725.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
When performing non-cascading cache invalidation, `recache` is called on the other cache entries which are dependent on the cache being invalidated. It leads to the the physical plans of those cache entries being re-compiled. For those cache entries, if the cache RDD has already been persisted, chances are there will be inconsistency between the data and the new plan. It can cause a correctness issue if the new plan's `outputPartitioning` or `outputOrdering` is different from the that of the actual data, and meanwhile the cache is used by another query that asks for specific `outputPartitioning` or `outputOrdering` which happens to match the new plan but not the actual data.
The fix is to keep the cache entry as it is if the data has been loaded, otherwise re-build the cache entry, with a new plan and an empty cache buffer.
## How was this patch tested?
Added UT.
Closes#23644 from maryannxue/spark-26708.
Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
1. Remove parameter `isReadPath`. The supported types of read/write should be the same.
2. Disallow reading `NullType` for ORC data source. In #21667 and #21389, it was supposed that ORC supports reading `NullType`, but can't write it. This doesn't make sense. I read docs and did some tests. ORC doesn't support `NullType`.
## How was this patch tested?
Unit tset
Closes#23639 from gengliangwang/supportDataType.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Spark replaces `CurrentTimestamp` with `CurrentBatchTimestamp`.
However, `CurrentBatchTimestamp` is `TimeZoneAwareExpression` while `CurrentTimestamp` isn't.
Without TimeZoneId, `CurrentBatchTimestamp` becomes unresolved and raises `UnresolvedException`.
Since `CurrentDate` is `TimeZoneAwareExpression`, there is no problem with `CurrentDate`.
This PR reverts the [previous patch](https://github.com/apache/spark/pull/23609) on `MicroBatchExecution` and fixes the root cause.
## How was this patch tested?
Pass the Jenkins with the updated test cases.
Closes#23660 from dongjoon-hyun/SPARK-26379.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Add verification of plan integrity with regards to special expressions being hosted only in supported operators. Specifically:
- `AggregateExpression`: should only be hosted in `Aggregate`, or indirectly in `Window`
- `WindowExpression`: should only be hosted in `Window`
- `Generator`: should only be hosted in `Generate`
This will help us catch errors in future optimizer rules that incorrectly hoist special expression out of their supported operator.
TODO: This PR actually caught a bug in the analyzer in the test case `SPARK-23957 Remove redundant sort from subquery plan(scalar subquery)` in `SubquerySuite`, where a `max()` aggregate function is hosted in a `Sort` operator in the analyzed plan, which is invalid. That test case is disabled in this PR.
SPARK-26741 has been opened to track the fix in the analyzer.
## How was this patch tested?
Added new test case in `OptimizerStructuralIntegrityCheckerSuite`
Closes#23658 from rednaxelafx/plan-integrity.
Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR targets to support Arrow optimization for conversion from R DataFrame to Spark DataFrame.
Like PySpark side, it falls back to non-optimization code path when it's unable to use Arrow optimization.
This can be tested as below:
```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
```r
collect(createDataFrame(mtcars))
```
### Requirements
- R 3.5.x
- Arrow package 0.12+
```bash
Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
```
**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.
### Benchmarks
**Shall**
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false
```
```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```
**R code**
```r
createDataFrame(mtcars) # Initializes
rdf <- read.csv("500000.csv")
test <- function() {
options(digits.secs = 6) # milliseconds
start.time <- Sys.time()
createDataFrame(rdf)
end.time <- Sys.time()
time.taken <- end.time - start.time
print(time.taken)
}
test()
```
**Data (350 MB):**
```r
object.size(read.csv("500000.csv"))
350379504 bytes
```
"500000 Records" http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/
**Results**
```
Time difference of 29.9468 secs
```
```
Time difference of 3.222129 secs
```
The performance improvement was around **950%**.
Actually, this PR improves around **1200%**+ because this PR includes a small optimization about regular R DataFrame -> Spark DatFrame. See https://github.com/apache/spark/pull/22954#discussion_r231847272
### Limitations:
For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.
In this case, we decide to fall back to non-optimization code path.
## How was this patch tested?
Small test was added.
I manually forced to set this optimization `true` for _all_ R tests and they were _all_ passed (with few of fallback warnings).
**TODOs:**
- [x] Draft codes
- [x] make the tests passed
- [x] make the CRAN check pass
- [x] Performance measurement
- [x] Supportability investigation (for instance types)
- [x] Wait for Arrow 0.12.0 release
- [x] Fix and match it to Arrow 0.12.0
Closes#22954 from HyukjinKwon/r-arrow-createdataframe.
Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When we read a hive table and create RDDs in `TableReader`, it'll throw exception `java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.TextInputFormat cannot be cast to org.apache.hadoop.mapred.InputFormat` if the input format class of the table is from mapreduce package.
Now we use NewHadoopRDD to deal with the new input format and keep HadoopRDD to the old one.
This PR is from #23506. We can reproduce this issue by executing the new test with the code in old version. When create a table with `org.apache.hadoop.mapreduce.....` input format, we will find the exception thrown in `org.apache.spark.rdd.HadoopRDD.getInputFormat(HadoopRDD.scala:190)`
## How was this patch tested?
Added a new test.
Closes#23559 from Deegue/fix-hadoopRDD.
Lead-authored-by: heguozi <zyzzxycj@gmail.com>
Co-authored-by: Yizhong Zhang <zyzzxycj@163.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR makes hardcoded configs about spark memory and storage to use `ConfigEntry` and put them in the config package.
## How was this patch tested?
Existing unit tests.
Closes#23623 from SongYadong/configEntry_for_mem_storage.
Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This PR fixes a bug where JSON schema inference attempts to convert every String value to a BigDecimal regardless of the setting of "prefersDecimal". With that bug, behavior is still correct, but performance is impacted.
This PR makes this conversion lazy, so it is only performed if prefersDecimal is set to true.
Using Spark with a single executor thread to infer the schema of a single-column, 100M row JSON file, the performance impact is as follows:
option | baseline | pr
-----|----|-----
inferTimestamp=_default_<br>prefersDecimal=_default_ | 12.5 minutes | 6.1 minutes |
inferTimestamp=false<br>prefersDecimal=_default_ | 6.5 minutes | 49 seconds |
inferTimestamp=false<br>prefersDecimal=true | 6.5 minutes | 6.5 minutes |
## How was this patch tested?
I ran JsonInferSchemaSuite and JsonSuite. Also, I ran manual tests to see performance impact (see above).
Closes#23653 from bersprockets/SPARK-26711_improved.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This patch proposes to fix issue on adding `current_timestamp` / `current_date` with streaming query.
The root reason is that Spark transforms `CurrentTimestamp`/`CurrentDate` to `CurrentBatchTimestamp` in MicroBatchExecution which makes transformed attributes not-yet-resolved. They will be resolved by IncrementalExecution.
(In ContinuousExecution, Spark doesn't allow using `current_timestamp` and `current_date` so it has been OK.)
It's OK for DataSource V1 sink because it simply leverages transformed logical plan and don't evaluate until they're resolved, but for DataSource V2 sink, Spark tries to extract the schema of transformed logical plan in prior to IncrementalExecution, and unresolved attributes will raise errors.
This patch fixes the issue via having separate pre-resolved logical plan to pass the schema to StreamingWriteSupport safely.
## How was this patch tested?
Added UT.
Closes#23609 from HeartSaVioR/SPARK-26379.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
There are ugly provided dependencies inside core for the following:
* Hive
* Kafka
In this PR I've extracted them out. This PR contains the following:
* Token providers are now loaded with service loader
* Hive token provider moved to hive project
* Kafka token provider extracted into a new project
## How was this patch tested?
Existing + newly added unit tests.
Additionally tested on cluster.
Closes#23499 from gaborgsomogyi/SPARK-26254.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
In the PR, I propose to remove the following methods from `DateTimeUtils`:
- `timestampAddInterval` and `stringToTimestamp` - used only in test suites
- `truncTimestamp`, `getSeconds`, `getMinutes`, `getHours` - those methods assume system default time zone. They are not used in Spark.
## How was this patch tested?
This was tested by `DateTimeUtilsSuite` and `UnsafeArraySuite`.
Closes#23643 from MaxGekk/unused-date-time-utils.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Noop data source for batch was added in [#23471](https://github.com/apache/spark/pull/23471).
In this PR I've added the streaming part.
## How was this patch tested?
Additional unit tests.
Closes#23631 from gaborgsomogyi/SPARK-26649.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
When reading from empty tables, the optimization `OptimizeMetadataOnlyQuery` may return wrong results:
```
sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)")
sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)")
sql("SELECT MAX(p1) FROM t")
```
The result is supposed to be `null`. However, with the optimization the result is `5`.
The rule is originally ported from https://issues.apache.org/jira/browse/HIVE-1003 in #13494. In Hive, the rule is disabled by default in a later release(https://issues.apache.org/jira/browse/HIVE-15397), due to the same problem.
It is hard to completely avoid the correctness issue. Because data sources like Parquet can be metadata-only. Spark can't tell whether it is empty or not without actually reading it. This PR disable the optimization by default.
## How was this patch tested?
Unit test
Closes#23635 from gengliangwang/optimizeMetadata.
Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Checkpoints of Dataframes currently do not show up in SQL UI. This PR fixes that by setting an execution id for the execution of the checkpoint by wrapping the checkpoint code with a `withAction`.
## How was this patch tested?
A unit test was added to DatasetSuite.
Closes#23636 from tomvanbussel/SPARK-26690.
Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
When a user passes a Stream to groupBy, ```CodegenSupport.consume``` ends up lazily generating ```inputVars``` from a Stream, since the field ```output``` will be a Stream. At the time ```output.zipWithIndex.map``` is called, conditions are correct. However, by the time the map operation actually executes, conditions are no longer appropriate. The closure used by the map operation ends up using a reference to the partially created ```inputVars```. As a result, a StackOverflowError occurs.
This PR ensures that ```inputVars``` is eagerly created while conditions are appropriate. It seems this was also an issue with the code path for creating ```inputVars``` from ```outputVars``` (SPARK-25767). I simply extended the solution for that code path to encompass both code paths.
## How was this patch tested?
SQL unit tests
new test
python tests
Closes#23617 from bersprockets/SPARK-26680_opt1.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
Updates the attempt ID used by FileFormatWriter. Tasks in stage attempts use the same task attempt number and could conflict. Using Spark's task attempt ID guarantees that Hadoop TaskAttemptID instances are unique.
## How was this patch tested?
Existing tests. Also validated that we no longer detect this failure case in our logs after deployment.
Closes#23608 from rdblue/SPARK-26682-fix-hadoop-task-attempt-id.
Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Fixed several places in CacheManager where a write lock was being held while running the query optimizer. This could cause a very lock block if the query optimization takes a long time. This builds on changes from [SPARK-26548] that fixed this issue for one specific case in the CacheManager.
gatorsmile This is very similar to the PR you approved last week.
## How was this patch tested?
Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.
Closes#23539 from DaveDeCaprio/cache-manager-locks.
Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Implement copy() for ColumnarArray
## How was this patch tested?
Updating test case to existing tests in ColumnVectorSuite
Closes#23569 from ayudovin/copy-for-columnArray.
Authored-by: ayudovin <a.yudovin6695@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR contains a minor change in `Cast$mayTruncate` that fixes its logic for bytes.
Right now, `mayTruncate(ByteType, LongType)` returns `false` while `mayTruncate(ShortType, LongType)` returns `true`. Consequently, `spark.range(1, 3).as[Byte]` and `spark.range(1, 3).as[Short]` behave differently.
Potentially, this bug can silently corrupt someone's data.
```scala
// executes silently even though Long is converted into Byte
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte]
.map(b => b - 1)
.show()
+-----+
|value|
+-----+
| -12|
| -11|
| -10|
| -9|
| -8|
| -7|
| -6|
| -5|
| -4|
| -3|
+-----+
// throws an AnalysisException: Cannot up cast `id` from bigint to smallint as it may truncate
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Short]
.map(s => s - 1)
.show()
```
## How was this patch tested?
This PR comes with a set of unit tests.
Closes#23632 from aokolnychyi/cast-fix.
Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
This adds a new pattern to recognize Ammonite REPL classes and return the correct scope.
## How was this patch tested?
Manually tested with Spark in an Ammonite session.
Closes#23607 from rdblue/SPARK-26681-support-ammonite-scopes.
Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose using of the `stringToDate` and `stringToTimestamp` methods in parsing JDBC lower/upper bounds of the partition column if it has `DateType` or `TimestampType`. Since those methods have been ported on Proleptic Gregorian calendar by #23512, the PR switches parsing of JDBC bounds of the partition column on the calendar as well.
## How was this patch tested?
This was tested by `JDBCSuite`.
Closes#23597 from MaxGekk/jdbc-parse-timestamp-bounds.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the master, GetArrayItem nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L236)
But, If input array size is constant and ordinal is foldable, we could make GetArrayItem nullability more precise. This pr added code to make `GetArrayItem` nullability more precise.
## How was this patch tested?
Added tests in `ComplexTypeSuite`.
Closes#23566 from maropu/GetArrayItemNullability.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The explain output of the Hive CTAS command, regardless of whether it's actually writing via Hive's SerDe or converted into using Spark's data source, would always show that it's using `InsertIntoHiveTable` because it's hardcoded.
e.g.
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
```
This CTAS is converted into using Spark's data source, but it still says `InsertIntoHiveTable` in the explain output.
It's better to show the actual class name of the writing command used. For the example above, it'd be:
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHadoopFsRelationCommand]
```
## How was this patch tested?
Added test case in `HiveExplainSuite`
Closes#23582 from rednaxelafx/fix-explain-1.
Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
The expressions `DayWeek`, `DayOfWeek`, `WeekDay` and `WeekOfYear` are changed to use Proleptic Gregorian calendar instead of the hybrid one (Julian+Gregorian). This was achieved by using Java 8 API for date/timestamp manipulation, in particular the `LocalDate` class.
Week of year calculation is performed according to ISO-8601. The first week of a week-based-year is the first Monday-based week of the standard ISO year that has at least 4 days in the new year (see https://docs.oracle.com/javase/8/docs/api/java/time/temporal/IsoFields.html).
## How was this patch tested?
The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.
Closes#23594 from MaxGekk/dayweek-gregorian.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.
## How was this patch tested?
Existing tests
Closes#23416 from kiszk/SPARK-26463.
Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Since 0.11.0, PyArrow supports to raise an error for unsafe cast ([PR](https://github.com/apache/arrow/pull/2504)). We should use it to raise a proper error for pandas udf users when such cast is detected.
Added a SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion` to disable Arrow safe type check.
## How was this patch tested?
Added test and manually test.
Closes#22807 from viirya/SPARK-25811.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Following https://github.com/apache/spark/pull/23086, this PR does the API refactor for micro-batch read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)
The major changes:
1. rename `XXXMicroBatchReadSupport` to `XXXMicroBatchReadStream`
2. implement `TableProvider`, `Table`, `ScanBuilder` and `Scan` for streaming sources
3. at the beginning of micro-batch streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.
followup:
support operator pushdown for stream sources
## How was this patch tested?
existing tests
Closes#23430 from cloud-fan/micro-batch.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The `fromString` and `fromJSON` methods of the `Literal` object are removed because they are not used.
Closes#23596Closes#23603 from MaxGekk/remove-literal-fromstring.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Correct some unit test mistakes.
## How was this patch tested?
N/A
Closes#23583 from 10110346/unused_symbol.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
The PR makes hardcoded `spark.unsafe` configs to use ConfigEntry and put them in the `config` package.
## How was this patch tested?
Existing UTs
Closes#23412 from kiszk/SPARK-26477.
Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
In the PR, I propose to make creation of typed Literals `TIMESTAMP` and `DATE` consistent to the `Cast` expression. More precisely, reusing the `Cast` expression in the type constructors. In this way, it allows:
- To use the same calendar in parsing methods
- To support the same set of timestamp/date patterns
For example, creating timestamp literal:
```sql
SELECT TIMESTAMP '2019-01-14 20:54:00.000'
```
behaves similarly as casting the string literal:
```sql
SELECT CAST('2019-01-14 20:54:00.000' AS TIMESTAMP)
```
## How was this patch tested?
This was tested by `SQLQueryTestSuite` as well as `ExpressionParserSuite`.
Closes#23541 from MaxGekk/timestamp-date-constructors.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
`DataWritingCommandExec` generates `cmd.nodeName` twice in its explain output, e.g. when running this query `spark.sql("create table foo stored as parquet as select id, id % 10 as cat1, id % 20 as cat2 from range(10)")`,
```
Execute OptimizedCreateHiveTableAsSelectCommand OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
+- *(1) Project [id#2L, (id#2L % 10) AS cat1#0L, (id#2L % 20) AS cat2#1L]
+- *(1) Range (0, 10, step=1, splits=8)
```
After the fix, it'll go back to normal:
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
+- *(1) Project [id#2L, (id#2L % 10) AS cat1#0L, (id#2L % 20) AS cat2#1L]
+- *(1) Range (0, 10, step=1, splits=8)
```
This duplication is introduced when this specialized `DataWritingCommandExec` was created in place of `ExecutedCommandExec`.
The former is a `UnaryExecNode` whose `children` include the physical plan of the query, and the `cmd` is picked up via `TreeNode.stringArgs` into the argument string. The duplication comes from: `DataWritingCommandExec.nodeName` is `s"Execute ${cmd.nodeName}"` while the argument string is `cmd.simpleString()` which also includes `cmd.nodeName`.
The latter didn't have that problem because it's a `LeafExecNode` with no children, and it declares the `cmd` as being a part of the `innerChildren` which is excluded from the argument string.
## How was this patch tested?
Manual testing of running the example above in a local Spark Shell.
Also added a new test case in `ExplainSuite`.
Closes#23579 from rednaxelafx/fix-explain.
Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Misc code cleanup from lgtm.com analysis. See comments below for details.
## How was this patch tested?
Existing tests.
Closes#23571 from srowen/SPARK-26640.
Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Try to make labels more obvious
"avg hash probe" avg hash probe bucket iterations
"partition pruning time (ms)" dynamic partition pruning time
"total number of files in the table" file count
"number of files that would be returned by partition pruning alone" file count after partition pruning
"total size of files in the table" file size
"size of files that would be returned by partition pruning alone" file size after partition pruning
"metadata time (ms)" metadata time
"aggregate time" time in aggregation build
"aggregate time" time in aggregation build
"time to construct rdd bc" time to build
"total time to remove rows" time to remove
"total time to update rows" time to update
Add proper metric type to some metrics:
"bytes of written output" written output - createSizeMetric
"metadata time" - createTimingMetric
"dataSize" - createSizeMetric
"collectTime" - createTimingMetric
"buildTime" - createTimingMetric
"broadcastTIme" - createTimingMetric
## How is this patch tested?
Existing tests.
Author: Stacy Kerkela <stacy.kerkeladatabricks.com>
Signed-off-by: Juliusz Sompolski <julekdatabricks.com>
Closes#23551 from juliuszsompolski/SPARK-26622.
Lead-authored-by: Juliusz Sompolski <julek@databricks.com>
Co-authored-by: Stacy Kerkela <stacy.kerkela@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The PR makes hardcoded `spark.shuffle` configs to use ConfigEntry and put them in the config package.
## How was this patch tested?
Existing unit tests
Closes#23550 from 10110346/ConfigEntry_shuffle.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to use *java.time* classes in `stringToDate` and `stringToTimestamp`. This switches the methods from the hybrid calendar (Gregorian+Julian) to Proleptic Gregorian calendar. And it should make the casting consistent to other Spark classes that converts textual representation of dates/timestamps to `DateType`/`TimestampType`.
## How was this patch tested?
The changes were tested by existing suites - `HashExpressionsSuite`, `CastSuite` and `DateTimeUtilsSuite`.
Closes#23512 from MaxGekk/utf8string-timestamp-parsing.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
Create a framework for file source V2 based on data source V2 API.
As a good example for demonstrating the framework, this PR also migrate ORC source. This is because ORC file source supports both row scan and columnar scan, and the implementation is simpler comparing with Parquet.
Note: Currently only read path of V2 API is done, this framework and migration are only for the read path.
Supports the following scan:
- Scan ColumnarBatch
- Scan UnsafeRow
- Push down filters
- Push down required columns
Not supported( due to the limitation of data source V2 API):
- Stats metrics
- Catalog table
- Writes
## How was this patch tested?
Unit test
Closes#23383 from gengliangwang/latest_orcV2.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The PR makes hardcoded configs below to use `ConfigEntry`.
* spark.kryo
* spark.kryoserializer
* spark.serializer
* spark.jars
* spark.files
* spark.submit
* spark.deploy
* spark.worker
This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).
## How was this patch tested?
Existing tests.
Closes#23532 from HeartSaVioR/SPARK-26466-v2.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
`SerializeFromObject` now keeps all serializer expressions for domain object even when only part of output attributes are used by top plan.
We should be able to prune unused serializers from `SerializeFromObject` in such case.
## How was this patch tested?
Added tests.
Closes#23562 from viirya/SPARK-26619.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
In the PR, I propose new built-in datasource with name `noop` which can be used in:
- benchmarking to avoid additional overhead of actions and unnecessary type conversions
- caching of datasets/dataframes
- producing other side effects as a consequence of row materialisations like uploading data to a IO caches.
## How was this patch tested?
Added a test to check that datasource rows are materialised.
Closes#23471 from MaxGekk/none-datasource.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
When a streaming query has multiple file streams, and there is a batch where one of the file streams dont have data in that batch, then if the query has to restart from that, it will throw the following error.
```
java.lang.IllegalStateException: batch 1 doesn't exist
at org.apache.spark.sql.execution.streaming.HDFSMetadataLog$.verifyBatchIds(HDFSMetadataLog.scala:300)
at org.apache.spark.sql.execution.streaming.FileStreamSourceLog.get(FileStreamSourceLog.scala:120)
at org.apache.spark.sql.execution.streaming.FileStreamSource.getBatch(FileStreamSource.scala:181)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:294)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:291)
at scala.collection.Iterator$class.foreach(Iterator.scala:891)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets(MicroBatchExecution.scala:291)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:178)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:251)
at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:61)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:175)
at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:169)
at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)
at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:205)
```
Existing `HDFSMetadata.verifyBatchIds` threw error whenever the `batchIds` list was empty. In the context of `FileStreamSource.getBatch` (where verify is called) and `FileStreamSourceLog` (subclass of `HDFSMetadata`), this is usually okay because, in a streaming query with one file stream, the `batchIds` can never be empty:
- A batch is planned only when the `FileStreamSourceLog` has seen new offset (that is, there are new data files).
- So `FileStreamSource.getBatch` will be called on X to Y where X will always be > Y. This calls internally`HDFSMetadata.verifyBatchIds (X+1, Y)` with X+1-Y ids.
For example.,`FileStreamSource.getBatch(4, 5)` will call `verify(batchIds = Seq(5), start = 5, end = 5)`. However, the invariant of X > Y is not true when there are two file stream sources, as a batch may be planned even when only one of the file streams has data. So one of the file stream may not have data, which can call `FileStreamSource.getBatch(X, X)` -> `verify(batchIds = Seq.empty, start = X+1, end = X)` -> failure.
Note that `FileStreamSource.getBatch(X, X)` gets called **only when restarting a query in a batch where a file source did not have data**. This is because in normal planning of batches, `MicroBatchExecution` avoids calling `FileStreamSource.getBatch(X, X)` when offset X has not changed. However, when restarting a stream at such a batch, `MicroBatchExecution.populateStartOffsets()` calls `FileStreamSource.getBatch(X, X)` (DataSource V1 hack to initialize the source with last known offsets) thus hitting this issue.
The minimum solution here is to skip verification when `FileStreamSource.getBatch(X, X)`.
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23557 from tdas/SPARK-26629.
Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
Adjust the batch write API to match the read API refactor after https://github.com/apache/spark/pull/23086
The doc with high-level ideas:
https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing
Basically it renames `BatchWriteSupportProvider` to `SupportsBatchWrite`, and make it extend `Table`. Renames `WriteSupport` to `Write`. It also cleans up some code as batch API is completed.
This PR also removes the test from https://github.com/apache/spark/pull/22688 . Now data source must return a table for read/write.
A few notes about future changes:
1. We will create `SupportsStreamingWrite` later for streaming APIs
2. We will create `SupportsBatchReplaceWhere`, `SupportsBatchAppend`, etc. for the new end-user write APIs. I think streaming APIs would remain to use `OutputMode`, and new end-user write APIs will apply to batch only, at least in the near future.
3. We will remove `SaveMode` from data source API: https://issues.apache.org/jira/browse/SPARK-26356
## How was this patch tested?
existing tests
Closes#23208 from cloud-fan/refactor-batch.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR contains benchmarks for `In` and `InSet` expressions. They cover literals of different data types and will help us to decide where to integrate the switch-based logic for bytes/shorts/ints.
As discussed in [PR-23171](https://github.com/apache/spark/pull/23171), one potential approach is to convert `In` to `InSet` if all elements are literals independently of data types and the number of elements. According to the results of this PR, we might want to keep the threshold for the number of elements. The if-else approach approach might be faster for some data types on a small number of elements (structs? arrays? small decimals?).
### byte / short / int / long
Unless the number of items is really big, `InSet` is slower than `In` because of autoboxing .
Interestingly, `In` scales worse on bytes/shorts than on ints/longs. For example, `InSet` starts to match the performance on around 50 bytes/shorts while this does not happen on the same number of ints/longs. This is a bit strange as shorts/bytes (e.g., `(byte) 1`, `(short) 2`) are represented as ints in the bytecode.
### float / double
Use cases on floats/doubles also suffer from autoboxing. Therefore, `In` outperforms `InSet` on 10 elements.
Similarly to shorts/bytes, `In` scales worse on floats/doubles than on ints/longs because the equality condition is more complicated (e.g., `java.lang.Float.isNaN(filter_valueArg_0) && java.lang.Float.isNaN(9.0F)) || filter_valueArg_0 == 9.0F`).
### decimal
The reason why we have separate benchmarks for small and large decimals is that Spark might use longs to represent decimals in some cases.
If this optimization happens, then `equals` will be nothing else as comparing longs. If this does not happen, Spark will create an instance of `scala.BigDecimal` and use it for comparisons. The latter is more expensive.
`Decimal$hashCode` will always use `scala.BigDecimal$hashCode` even if the number is small enough to fit into a long variable. As a consequence, we see that use cases on small decimals are faster with `In` as they are using long comparisons under the hood. Large decimal values are always faster with `InSet`.
### string
`UTF8String$equals` is not cheap. Therefore, `In` does not really outperform `InSet` as in previous use cases.
### timestamp / date
Under the hood, timestamp/date values will be represented as long/int values. So, `In` allows us to avoid autoboxing.
### array
Arrays are working as expected. `In` is faster on 5 elements while `InSet` is faster on 15 elements. The benchmarks are using `UnsafeArrayData`.
### struct
`InSet` is always faster than `In` for structs. These benchmarks use `GenericInternalRow`.
Closes#23291 from aokolnychyi/spark-26203.
Lead-authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This PR reverts #22938 per discussion in #23325Closes#23325Closes#23543 from MaxGekk/return-nulls-from-json-parser.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to switch on `TimestampFormatter`/`DateFormatter` in casting dates/timestamps to strings. The changes should make the date/timestamp casting consistent to JSON/CSV datasources and time-related functions like `to_date`, `to_unix_timestamp`/`from_unixtime`.
Local formatters are moved out from `DateTimeUtils` to where they are actually used. It allows to avoid re-creation of new formatter instance per-each call. Another reason is to have separate parser for `PartitioningUtils` because default parsing pattern cannot be used (expected optional section `[.S]`).
## How was this patch tested?
It was tested by `DateTimeUtilsSuite`, `CastSuite` and `JDBC*Suite`.
Closes#23391 from MaxGekk/thread-local-date-format.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Make sure broadcast hint is applied to partitioned tables.
## How was this patch tested?
- A new unit test in PruneFileSourcePartitionsSuite
- Unit test suites touched by SPARK-14581: JoinOptimizationSuite, FilterPushdownSuite, ColumnPruningSuite, and PruneFiltersSuite
Closes#23507 from jzhuge/SPARK-26576.
Closes#23530 from jzhuge/SPARK-26576-master.
Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This is to fix a bug in #23036 that would cause a join hint to be applied on node it is not supposed to after join reordering. For example,
```
val join = df.join(df, "id")
val broadcasted = join.hint("broadcast")
val join2 = join.join(broadcasted, "id").join(broadcasted, "id")
```
There should only be 2 broadcast hints on `join2`, but after join reordering there would be 4. It is because the hint application in join reordering compares the attribute set for testing relation equivalency.
Moreover, it could still be problematic even if the child relations were used in testing relation equivalency, due to the potential exprId conflict in nested self-join.
As a result, this PR simply reverts the join reorder hint behavior change introduced in #23036, which means if a join hint is present, the join node itself will not participate in the join reordering, while the sub-joins within its children still can.
## How was this patch tested?
Added new tests
Closes#23524 from maryannxue/query-hint-followup-2.
Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
When creating some unsafe projections, Spark rebuilds the map of schema attributes once for each expression in the projection. Some file format readers create one unsafe projection per input file, others create one per task. ProjectExec also creates one unsafe projection per task. As a result, for wide queries on wide tables, Spark might build the map of schema attributes hundreds of thousands of times.
This PR changes two functions to reuse the same AttributeSeq instance when creating BoundReference objects for each expression in the projection. This avoids the repeated rebuilding of the map of schema attributes.
### Benchmarks
The time saved by this PR depends on size of the schema, size of the projection, number of input files (or number of file splits), number of tasks, and file format. I chose a couple of example cases.
In the following tests, I ran the query
```sql
select * from table where id1 = 1
```
Matching rows are about 0.2% of the table.
#### Orc table 6000 columns, 500K rows, 34 input files
baseline | pr | improvement
----|----|----
1.772306 min | 1.487267 min | 16.082943%
#### Orc table 6000 columns, 500K rows, *17* input files
baseline | pr | improvement
----|----|----
1.656400 min | 1.423550 min | 14.057595%
#### Orc table 60 columns, 50M rows, 34 input files
baseline | pr | improvement
----|----|----
0.299878 min | 0.290339 min | 3.180926%
#### Parquet table 6000 columns, 500K rows, 34 input files
baseline | pr | improvement
----|----|----
1.478306 min | 1.373728 min | 7.074165%
Note: The parquet reader does not create an unsafe projection. However, the filter operation in the query causes the planner to add a ProjectExec, which does create an unsafe projection for each task. So these results have nothing to do with Parquet itself.
#### Parquet table 60 columns, 50M rows, 34 input files
baseline | pr | improvement
----|----|----
0.245006 min | 0.242200 min | 1.145099%
#### CSV table 6000 columns, 500K rows, 34 input files
baseline | pr | improvement
----|----|----
2.390117 min | 2.182778 min | 8.674844%
#### CSV table 60 columns, 50M rows, 34 input files
baseline | pr | improvement
----|----|----
1.520911 min | 1.510211 min | 0.703526%
## How was this patch tested?
SQL unit tests
Python core and SQL test
Closes#23392 from bersprockets/norebuild.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
The SQL config `spark.sql.legacy.timeParser.enabled` was removed by https://github.com/apache/spark/pull/23495. The PR cleans up the SQL migration guide and the comment for `UnixTimestamp`.
Closes#23529 from MaxGekk/get-rid-off-legacy-parser-followup.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
If users set equivalent values to spark.network.timeout and spark.executor.heartbeatInterval, they get the following message:
```
java.lang.IllegalArgumentException: requirement failed: The value of spark.network.timeout=120s must be no less than the value of spark.executor.heartbeatInterval=120s.
```
But it's misleading since it can be read as they could be equal. So this PR replaces "no less than" with "greater than". Also, it fixes similar inconsistencies found in MLlib and SQL components.
## How was this patch tested?
Ran Spark with equivalent values for them manually and confirmed that the revised message was displayed.
Closes#23488 from sekikn/SPARK-26564.
Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
When determining CatalystType for postgres columns with type `numeric[]` set the type of array element to `DecimalType(38, 18)` instead of `DecimalType(0,0)`.
## How was this patch tested?
Tested with modified `org.apache.spark.sql.jdbc.JDBCSuite`.
Ran the `PostgresIntegrationSuite` manually.
Closes#23456 from a-shkarupin/postgres_numeric_array.
Lead-authored-by: Oleksii Shkarupin <a.shkarupin@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
The vote of final release of `branch-2.2` passed and the branch goes EOL. This PR removes Spark 2.2.x from the testing coverage.
## How was this patch tested?
Pass the Jenkins.
Closes#23526 from dongjoon-hyun/SPARK-26607.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Fix race condition where streams can have unexpected conf values.
New streaming queries should run with isolated SparkSessions so that they aren't affected by conf updates after they are started. In StreamExecution, the parent SparkSession is cloned and used to run each batch, but this cloning happens in a separate thread and may happen after DataStreamWriter.start() returns. If a stream is started and a conf key is set immediately after, the stream is likely to have the new value.
## How was this patch tested?
New unit test that fails prior to the production change and passes with it.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23513 from mukulmurthy/26586.
Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
Schema pruning has errors when selecting one complex field and having is not null predicate on another one:
```scala
val query = sql("select * from contacts")
.where("name.middle is not null")
.select(
"id",
"name.first",
"name.middle",
"name.last"
)
.where("last = 'Jones'")
.select(count("id"))
```
```
java.lang.IllegalArgumentException: middle does not exist. Available: last
[info] at org.apache.spark.sql.types.StructType.$anonfun$fieldIndex$1(StructType.scala:303)
[info] at scala.collection.immutable.Map$Map1.getOrElse(Map.scala:119)
[info] at org.apache.spark.sql.types.StructType.fieldIndex(StructType.scala:302)
[info] at org.apache.spark.sql.execution.ProjectionOverSchema.$anonfun$getProjection$6(ProjectionOverSchema.scala:58)
[info] at scala.Option.map(Option.scala:163)
[info] at org.apache.spark.sql.execution.ProjectionOverSchema.getProjection(ProjectionOverSchema.scala:56)
[info] at org.apache.spark.sql.execution.ProjectionOverSchema.unapply(ProjectionOverSchema.scala:32)
[info] at org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaPruning$$anonfun$$nestedInanonfun$buildNewProjection$1$1.applyOrElse(Parque
tSchemaPruning.scala:153)
```
## How was this patch tested?
Added tests.
Closes#23474 from viirya/SPARK-26551.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
The PR makes hardcoded configs below to use `ConfigEntry`.
* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl
This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).
## How was this patch tested?
Existing tests.
Closes#23423 from HeartSaVioR/SPARK-26466.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Per discussion in #23391 (comment) this proposes to just remove the old pre-Spark-3 time parsing behavior.
This is a rebase of https://github.com/apache/spark/pull/23411
## How was this patch tested?
Existing tests.
Closes#23495 from srowen/SPARK-26503.2.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/22732 , we tried our best to keep the behavior of Scala UDF unchanged in Spark 2.4.
However, since Spark 3.0, Scala 2.12 is the default. The trick that was used to keep the behavior unchanged doesn't work with Scala 2.12.
This PR proposes to remove the Scala 2.11 hack, as it's not useful.
## How was this patch tested?
existing tests.
Closes#23498 from cloud-fan/udf.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR aims to remove internal ORC configuration to simplify the code path for Spark 3.0.0. This removes the configuration `spark.sql.orc.copyBatchToSpark` and related ORC codes including tests and benchmarks.
## How was this patch tested?
Pass the Jenkins with the reduced test coverage.
Closes#23503 from dongjoon-hyun/SPARK-26584.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Remove spark.memory.useLegacyMode and StaticMemoryManager. Update tests that used the StaticMemoryManager to equivalent use of UnifiedMemoryManager.
## How was this patch tested?
Existing tests, with modifications to make them work with a different mem manager.
Closes#23457 from srowen/SPARK-26539.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/18576
The newly added rule `UpdateNullabilityInAttributeReferences` does the same thing the `FixNullability` does, we only need to keep one of them.
This PR removes `UpdateNullabilityInAttributeReferences`, and use `FixNullability` to replace it. Also rename it to `UpdateAttributeNullability`
## How was this patch tested?
existing tests
Closes#23390 from cloud-fan/nullable.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
Added a cache for java.time.format.DateTimeFormatter instances with keys consist of pattern and locale. This should allow to avoid parsing of timestamp/date patterns each time when new instance of `TimestampFormatter`/`DateFormatter` is created.
## How was this patch tested?
By existing test suites `TimestampFormatterSuite`/`DateFormatterSuite` and `JsonFunctionsSuite`/`JsonSuite`.
Closes#23462 from MaxGekk/time-formatter-caching.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Allow multiple spark.sql.extensions to be specified in the
configuration.
## How was this patch tested?
New tests are added.
Closes#23398 from jamisonbennett/SPARK-26493.
Authored-by: Jamison Bennett <jamison.bennett@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is to fix a bug in https://github.com/apache/spark/pull/23036, which would lead to an exception in case of two consecutive hints.
## How was this patch tested?
Added a new test.
Closes#23501 from maryannxue/query-hint-followup.
Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/23043 , we introduced a behavior change: Spark users are not able to distinguish 0.0 and -0.0 anymore.
This PR proposes an alternative fix to the original bug, to retain the difference between 0.0 and -0.0 inside Spark.
The idea is, we can rewrite the window partition key, join key and grouping key during logical phase, to normalize the special floating numbers. Thus only operators care about special floating numbers need to pay the perf overhead, and end users can distinguish -0.0.
## How was this patch tested?
existing test
Closes#23388 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Refactor ExternalAppendOnlyUnsafeRowArrayBenchmark to use main method.
## How was this patch tested?
Manually tested and regenerated results.
Please note that `spark.memory.debugFill` setting has a huge impact on this benchmark. Since it is set to true by default when running the benchmark from SBT, we need to disable it:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql;set javaOptions in Test += \"-Dspark.memory.debugFill=false\";test:runMain org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark"
```
Closes#22617 from peter-toth/SPARK-25484.
Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Peter Toth <ptoth@hortonworks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Currently Spark table maintains Hive catalog storage format, so that Hive client can read it. In `HiveSerDe.scala`, Spark uses a mapping from its data source to HiveSerde. The mapping is old, we need to update with latest canonical name of Parquet and Orc FileFormat.
Otherwise the following queries will result in wrong Serde value in Hive table(default value `org.apache.hadoop.mapred.SequenceFileInputFormat`), and Hive client will fail to read the output table:
```
df.write.format("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat").saveAsTable(..)
```
```
df.write.format("org.apache.spark.sql.execution.datasources.orc.OrcFileFormat").saveAsTable(..)
```
This minor PR is to fix the mapping.
## How was this patch tested?
Unit test.
Closes#23491 from gengliangwang/fixHiveSerdeMap.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.
This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.
The new filter is added at the end of the filter chain, because authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.
As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.
Tested with added unit tests and in a history server with SPNEGO auth
configured.
Closes#23302 from vanzin/SPARK-24522.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
## What changes were proposed in this pull request?
Fixing leap year calculations for date operators (year/month/dayOfYear) where the Julian calendars are used (before 1582-10-04). In a Julian calendar every years which are multiples of 4 are leap years (there is no extra exception for years multiples of 100).
## How was this patch tested?
With a unit test ("SPARK-26002: correct day of year calculations for Julian calendar years") which focuses to these corner cases.
Manually:
```
scala> sql("select year('1500-01-01')").show()
+------------------------------+
|year(CAST(1500-01-01 AS DATE))|
+------------------------------+
| 1500|
+------------------------------+
scala> sql("select dayOfYear('1100-01-01')").show()
+-----------------------------------+
|dayofyear(CAST(1100-01-01 AS DATE))|
+-----------------------------------+
| 1|
+-----------------------------------+
```
Closes#23000 from attilapiros/julianOffByDays.
Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
For Scala UDF, when checking input nullability, we will skip inputs with type `Any`, and only check the inputs that provide nullability info.
We should do the same for checking input types.
## How was this patch tested?
new tests
Closes#23275 from cloud-fan/udf.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
The PR https://github.com/apache/spark/pull/23446 happened to introduce a behaviour change - empty dataframes can't be read anymore from underscore files. It looks controversial to allow or disallow this case so this PR targets to fix to issue warning instead of throwing an exception to be more conservative.
**Before**
```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
file:/.../_tmp1;
at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:651)
at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:635)
... 49 elided
scala> spark.read.text("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
file:/.../_tmp1;
at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:723)
at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:695)
... 49 elided
```
**After**
```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
19/01/07 15:14:43 WARN DataSource: All paths were ignored:
file:/.../_tmp
file:/.../_tmp1
+---+
| a|
+---+
+---+
scala> spark.read.text("_tmp*").show()
19/01/07 15:14:51 WARN DataSource: All paths were ignored:
file:/.../_tmp
file:/.../_tmp1
+-----+
|value|
+-----+
+-----+
```
## How was this patch tested?
Manually tested as above.
Closes#23481 from HyukjinKwon/SPARK-26339.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The PR makes hardcoded `spark.test` and `spark.testing` configs to use `ConfigEntry` and put them in the config package.
## How was this patch tested?
existing UTs
Closes#23413 from mgaido91/SPARK-26491.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
The existing query hint implementation relies on a logical plan node `ResolvedHint` to store query hints in logical plans, and on `Statistics` in physical plans. Since `ResolvedHint` is not really a logical operator and can break the pattern matching for existing and future optimization rules, it is a issue to the Optimizer as the old `AnalysisBarrier` was to the Analyzer.
Given the fact that all our query hints are either 1) a join hint, i.e., broadcast hint; or 2) a re-partition hint, which is indeed an operator, we only need to add a hint field on the Join plan and that will be a good enough solution for the current hint usage.
This PR is to let `Join` node have a hint for its left sub-tree and another hint for its right sub-tree and each hint is a merged result of all the effective hints specified in the corresponding sub-tree. The "effectiveness" of a hint, i.e., whether that hint should be propagated to the `Join` node, is currently consistent with the hint propagation rules originally implemented in the `Statistics` approach. Note that the `ResolvedHint` node still has to live through the analysis stage because of the `Dataset` interface, but it will be got rid of and moved to the `Join` node in the "pre-optimization" stage.
This PR also introduces a change in how hints work with join reordering. Before this PR, hints would stop join reordering. For example, in "a.join(b).join(c).hint("broadcast").join(d)", the broadcast hint would stop d from participating in the cost-based join reordering while still allowing reordering from under the hint node. After this PR, though, the broadcast hint will not interfere with join reordering at all, and after reordering if a relation associated with a hint stays unchanged or equivalent to the original relation, the hint will be retained, otherwise will be discarded. For example, the original plan is like "a.join(b).hint("broadcast").join(c).hint("broadcast").join(d)", thus the join order is "a JOIN b JOIN c JOIN d". So if after reordering the join order becomes "a JOIN b JOIN (c JOIN d)", the plan will be like "a.join(b).hint("broadcast").join(c.join(d))"; but if after reordering the join order becomes "a JOIN c JOIN b JOIN d", the plan will be like "a.join(c).join(b).hint("broadcast").join(d)".
## How was this patch tested?
Added new tests.
Closes#23036 from maryannxue/query-hint.
Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
When passing wrong url to jdbc then It would throw IllegalArgumentException instead of NPE.
### How was this patch tested?
Adding test case to Existing tests in JDBCSuite
Closes#23464 from ayudovin/fixing-npe.
Authored-by: ayudovin <a.yudovin6695@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
`StreamingReadSupport` is designed to be a `package` interface. Mockito seems to complain during `Maven` testing. This doesn't fail in `sbt` and IntelliJ. For mock-testing purpose, this PR makes it `public` interface and adds explicit comments like `public interface ReadSupport`
```scala
EpochCoordinatorSuite:
*** RUN ABORTED ***
java.lang.IllegalAccessError: tried to
access class org.apache.spark.sql.sources.v2.reader.streaming.StreamingReadSupport
from class org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338
at org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338.<clinit>(Unknown Source)
at sun.reflect.GeneratedSerializationConstructorAccessor632.newInstance(Unknown Source)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.objenesis.instantiator.sun.SunReflectionFactoryInstantiator.newInstance(SunReflectionFactoryInstantiator.java:48)
at org.objenesis.ObjenesisBase.newInstance(ObjenesisBase.java:73)
at org.mockito.internal.creation.instance.ObjenesisInstantiator.newInstance(ObjenesisInstantiator.java:19)
at org.mockito.internal.creation.bytebuddy.SubclassByteBuddyMockMaker.createMock(SubclassByteBuddyMockMaker.java:47)
at org.mockito.internal.creation.bytebuddy.ByteBuddyMockMaker.createMock(ByteBuddyMockMaker.java:25)
at org.mockito.internal.util.MockUtil.createMock(MockUtil.java:35)
at org.mockito.internal.MockitoCore.mock(MockitoCore.java:69)
```
## How was this patch tested?
Pass the Jenkins with Maven build
Closes#23463 from dongjoon-hyun/SPARK-26536-2.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
The `toHiveString()` and `toHiveStructString` methods were removed from `HiveUtils` because they have been already implemented in `HiveResult`. One related test was moved to `HiveResultSuite`.
## How was this patch tested?
By tests from `hive-thriftserver`.
Closes#23466 from MaxGekk/dedup-hive-result-string.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
My pull request #23288 was resolved and merged to master, but it turned out later that my change breaks another regression test. Because we cannot reopen pull request, I create a new pull request here.
Commit 92934b4 is only change after pull request #23288.
`CheckFileExist` was avoided at 239cfa4 after discussing #23288 (comment).
But, that change turned out to be wrong because we should not check if argument checkFileExist is false.
Test 27e42c1de5/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala (L2555)
failed when we avoided checkFileExist, but now successed after commit 92934b4 .
## How was this patch tested?
Both of below tests were passed.
```
testOnly org.apache.spark.sql.execution.datasources.csv.CSVSuite
testOnly org.apache.spark.sql.SQLQuerySuite
```
Closes#23446 from KeiichiHirobe/SPARK-26339.
Authored-by: Hirobe Keiichi <keiichi_hirobe@forcia.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Address SPARK-26548, in Spark 2.4.0, the CacheManager holds a write lock while computing the executedPlan for a cached logicalPlan. In some cases with very large query plans this can be an expensive operation, taking minutes to run. The entire cache is blocked during this time. This PR changes that so the writeLock is only obtained after the executedPlan is generated, this reduces the time the lock is held to just the necessary time when the shared data structure is being updated.
gatorsmile and cloud-fan - You can committed patches in this area before. This is a small incremental change.
## How was this patch tested?
Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.
Closes#23469 from DaveDeCaprio/optimizer-unblocked.
Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The truth table comment in EqualNullSafe incorrectly marked FALSE results as UNKNOWN.
## How was this patch tested?
N/A
Closes#23461 from rednaxelafx/fix-typo.
Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Added new JSON option `inferTimestamp` (`true` by default) to control inferring of `TimestampType` from string values.
## How was this patch tested?
Add new UT to `JsonInferSchemaSuite`.
Closes#23455 from MaxGekk/json-infer-time-followup.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
While backporting the patch to 2.4/2.3, I realized that the patch introduces unneeded imports (probably leftovers from intermediate changes). This PR removes the useless import.
## How was this patch tested?
NA
Closes#23451 from mgaido91/SPARK-26078_FOLLOWUP.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.
- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
saslHandler.channelInactive(null);
- verify(handler).channelInactive(any(TransportClient.class));
+ verify(handler).channelInactive(isNull());
```
- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```
## How was this patch tested?
Pass the Jenkins with the existing tests.
Closes#23452 from dongjoon-hyun/SPARK-26536.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Increase test memory to avoid OOM in TimSort-related tests.
## How was this patch tested?
Existing tests.
Closes#23425 from srowen/SPARK-26306.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Test case in SPARK-10316 is used to make sure non-deterministic `Filter` won't be pushed through `Project`
But in current code base this test case can't cover this purpose.
Change LogicalRDD to HadoopFsRelation can fix this issue.
## How was this patch tested?
Modified test pass.
Closes#23440 from LinhongLiu/fix-test.
Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently OrcColumnarBatchReader returns all the partition column values in the batch read.
In data source V2, we can improve it by returning the required partition column values only.
This PR is part of https://github.com/apache/spark/pull/23383 . As cloud-fan suggested, create a new PR to make review easier.
Also, this PR doesn't improve `OrcFileFormat`, since in the method `buildReaderWithPartitionValues`, the `requiredSchema` filter out all the partition columns, so we can't know which partition column is required.
## How was this patch tested?
Unit test
Closes#23387 from gengliangwang/refactorOrcColumnarBatch.
Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
`testExactCaseQueryPruning` and `testMixedCaseQueryPruning` don't need to set up `PARQUET_VECTORIZED_READER_ENABLED` config. Because `withMixedCaseData` will run against both Spark vectorized reader and Parquet-mr reader.
## How was this patch tested?
Existing test.
Closes#23427 from viirya/fix-parquet-schema-pruning-test.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to move `hiveResultString()` out of `QueryExecution` and put it to a separate object.
Closes#23409 from MaxGekk/hive-result-string.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
This PR fixes `pivot(Column)` can accepts `collection.mutable.WrappedArray`.
Note that we return `collection.mutable.WrappedArray` from `ArrayType`, and `Literal.apply` doesn't support this.
We can unwrap the array and use it for type dispatch.
```scala
val df = Seq(
(2, Seq.empty[String]),
(2, Seq("a", "x")),
(3, Seq.empty[String]),
(3, Seq("a", "x"))).toDF("x", "s")
df.groupBy("x").pivot("s").count().show()
```
Before:
```
Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray()
java.lang.RuntimeException: Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray()
at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:80)
at org.apache.spark.sql.RelationalGroupedDataset.$anonfun$pivot$2(RelationalGroupedDataset.scala:427)
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:39)
at scala.collection.TraversableLike.map(TraversableLike.scala:237)
at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
at scala.collection.AbstractTraversable.map(Traversable.scala:108)
at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:425)
at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:406)
at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:317)
at org.apache.spark.sql.DataFramePivotSuite.$anonfun$new$1(DataFramePivotSuite.scala:341)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
```
After:
```
+---+---+------+
| x| []|[a, x]|
+---+---+------+
| 3| 1| 1|
| 2| 1| 1|
+---+---+------+
```
## How was this patch tested?
Manually tested and unittests were added.
Closes#23349 from HyukjinKwon/SPARK-26403.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
`DataSourceScanExec` overrides "wrong" `treeString` method without `append`. In the PR, I propose to make `treeString`s **final** to prevent such mistakes in the future. And removed the `treeString` and `verboseString` since they both use `simpleString` with reduction.
## How was this patch tested?
It was tested by `DataSourceScanExecRedactionSuite`
Closes#23431 from MaxGekk/datasource-scan-exec-followup.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.metric.SQLMetricsSuite`, there's a test case named "WholeStageCodegen metrics". However, it is executed with whole-stage codegen disabled. This PR fixes this by enable whole-stage codegen for this test case.
## How was this patch tested?
Tested locally using exiting test cases.
Closes#23224 from seancxmao/codegen-metrics.
Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com).
[lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project.
This PR addresses the following warnings:
- Result of multiplication cast to wider type
- Implicit narrowing conversion in compound assignment
- Boxed variable is never null
- Useless null check
NOTE: `Potential input resource leak` looks false positive for now.
## How was this patch tested?
Existing UTs
Closes#23420 from kiszk/SPARK-26508.
Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Default timestamp pattern defined in `JSONOptions` doesn't allow saving/loading timestamps with time zones of seconds precision. Because of that, the round trip test failed for timestamps before 1582. In the PR, I propose to extend zone offset section from `XXX` to `XXXXX` which should allow to save/load zone offsets like `-07:52:48`.
## How was this patch tested?
It was tested by `JsonHadoopFsRelationSuite` and `TimestampFormatterSuite`.
Closes#23417 from MaxGekk/hadoopfsrelationtest-new-formatter.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When we operate as below:
`
0: jdbc:hive2://xxx/> create function funnel_analysis as 'com.xxx.hive.extend.udf.UapFunnelAnalysis';
`
`
0: jdbc:hive2://xxx/> select funnel_analysis(1,",",1,'');
Error: org.apache.spark.sql.AnalysisException: Undefined function: 'funnel_analysis'. This function is neither a registered temporary function nor a permanent function registered in the database 'xxx'.; line 1 pos 7 (state=,code=0)
`
`
0: jdbc:hive2://xxx/> describe function funnel_analysis;
+-----------------------------------------------------------+--+
| function_desc |
+-----------------------------------------------------------+--+
| Function: xxx.funnel_analysis |
| Class: com.xxx.hive.extend.udf.UapFunnelAnalysis |
| Usage: N/A. |
+-----------------------------------------------------------+--+
`
We can see describe funtion will get right information,but when we actually use this funtion,we will get an undefined exception.
Which is really misleading,the real cause is below:
`
No handler for Hive UDF 'com.xxx.xxx.hive.extend.udf.UapFunnelAnalysis': java.lang.IllegalStateException: Should not be called directly;
at org.apache.hadoop.hive.ql.udf.generic.GenericUDTF.initialize(GenericUDTF.java:72)
at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector$lzycompute(hiveUDFs.scala:204)
at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector(hiveUDFs.scala:204)
at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema$lzycompute(hiveUDFs.scala:212)
at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema(hiveUDFs.scala:212)
`
This patch print the actual failure for quick debugging.
## How was this patch tested?
UT
Closes#21790 from caneGuy/zhoukang/print-warning1.
Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
…Type
## What changes were proposed in this pull request?
Modifed JdbcUtils.makeGetter to handle ByteType.
## How was this patch tested?
Added a new test to JDBCSuite that maps ```TINYINT``` to ```ByteType```.
Closes#23400 from twdsilva/tiny_int_support.
Authored-by: Thomas D'Silva <tdsilva@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
The current `SelectedField` extractor is somewhat complicated and it seems to be handling cases that should be handled automatically:
- `GetArrayItem(child: GetStructFieldObject())`
- `GetArrayStructFields(child: GetArrayStructFields())`
- `GetMap(value: GetStructFieldObject())`
This PR removes those cases and simplifies the extractor by passing down the data type instead of a field.
## How was this patch tested?
Existing tests.
Closes#23397 from hvanhovell/SPARK-26495.
Authored-by: Herman van Hovell <hvanhovell@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
As the description in SPARK-26339, spark.read behavior is very confusing when reading files that start with underscore, fix this by throwing exception which message is "Path does not exist".
## How was this patch tested?
manual tests.
Both of codes below throws exception which message is "Path does not exist".
```
spark.read.csv("/home/forcia/work/spark/_test.csv")
spark.read.schema("test STRING, number INT").csv("/home/forcia/work/spark/_test.csv")
```
Closes#23288 from KeiichiHirobe/SPARK-26339.
Authored-by: Hirobe Keiichi <keiichi_hirobe@forcia.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Proposed new class `StringConcat` for converting a sequence of strings to string with one memory allocation in the `toString` method. `StringConcat` replaces `StringBuilderWriter` in methods of dumping of Spark plans and codegen to strings.
All `Writer` arguments are replaced by `String => Unit` in methods related to Spark plans stringification.
## How was this patch tested?
It was tested by existing suites `QueryExecutionSuite`, `DebuggingSuite` as well as new tests for `StringConcat` in `StringUtilsSuite`.
Closes#23406 from MaxGekk/rope-plan.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
#20560/[SPARK-23375](https://issues.apache.org/jira/browse/SPARK-23375) introduced an optimizer rule to eliminate redundant Sort. For a test case named "Sort metrics" in `SQLMetricsSuite`, because range is already sorted, sort is removed by the `RemoveRedundantSorts`, which makes this test case meaningless.
This PR modifies the query for testing Sort metrics and checks Sort exists in the plan.
## How was this patch tested?
Modify the existing test case.
Closes#23258 from seancxmao/sort-metrics.
Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Similar with https://github.com/apache/spark/pull/21446. Looks random string is not quite safe as a directory name.
```scala
scala> val prefix = Random.nextString(10); val dir = new File("/tmp", "del_" + prefix + "-" + UUID.randomUUID.toString); dir.mkdirs()
prefix: String = 窽텘⒘駖ⵚ駢⡞Ρ닋
dir: java.io.File = /tmp/del_窽텘⒘駖ⵚ駢⡞Ρ닋-a3f99855-c429-47a0-a108-47bca6905745
res40: Boolean = false // nope, didn't like this one
```
## How was this patch tested?
Unit test was added, and manually.
Closes#23405 from HyukjinKwon/SPARK-26496.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This PR fixes the codegen bug introduced by #23358 .
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.11/158/
```
Line 44, Column 93: A method named "apply" is not declared in any enclosing class
nor any supertype, nor through a static import
```
## How was this patch tested?
Manual. `DateExpressionsSuite` should be passed with Scala-2.11.
Closes#23394 from dongjoon-hyun/SPARK-26424.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Currently the `AttributeReference.withMetadata` method have return type `Attribute`, the rest of with methods in the `AttributeReference` return type are `AttributeReference`, as the [spark-25892](https://issues.apache.org/jira/browse/SPARK-25892?jql=project%20%3D%20SPARK%20AND%20component%20in%20(ML%2C%20PySpark%2C%20SQL)) mentioned.
This PR will change `AttributeReference.withMetadata` method's return type from `Attribute` to `AttributeReference`.
## How was this patch tested?
Run all `sql/test,` `catalyst/test` and `org.apache.spark.sql.execution.streaming.*`
Closes#22918 from kevinyu98/spark-25892.
Authored-by: Kevin Yu <qyu@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
In the PR, I propose to add `maxFields` parameter to all functions involved in creation of textual representation of spark plans such as `simpleString` and `verboseString`. New parameter restricts number of fields converted to truncated strings. Any elements beyond the limit will be dropped and replaced by a `"... N more fields"` placeholder. The threshold is bumped up to `Int.MaxValue` for `toFile()`.
## How was this patch tested?
Added a test to `QueryExecutionSuite` which checks `maxFields` impacts on number of truncated fields in `LocalRelation`.
Closes#23159 from MaxGekk/to-file-max-fields.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
Spark SQL doesn't support creating partitioned table using Hive CTAS in SQL syntax. However it is supported by using DataFrameWriter API.
```scala
val df = Seq(("a", 1)).toDF("part", "id")
df.write.format("hive").partitionBy("part").saveAsTable("t")
```
Hive begins to support this syntax in newer version: https://issues.apache.org/jira/browse/HIVE-20241:
```
CREATE TABLE t PARTITIONED BY (part) AS SELECT 1 as id, "a" as part
```
This patch adds this support to SQL syntax.
## How was this patch tested?
Added tests.
Closes#23376 from viirya/hive-ctas-partitioned-table.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to switch the `DateFormatClass`, `ToUnixTimestamp`, `FromUnixTime`, `UnixTime` on java.time API for parsing/formatting dates and timestamps. The API has been already implemented by the `Timestamp`/`DateFormatter` classes. One of benefit is those classes support parsing timestamps with microsecond precision. Old behaviour can be switched on via SQL config: `spark.sql.legacy.timeParser.enabled` (`false` by default).
## How was this patch tested?
It was tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite`, `JsonSuite`, `CsvSuite`, `SQLQueryTestSuite` as well as PySpark tests.
Closes#23358 from MaxGekk/new-time-cast.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
fix ExpresionInfo assert error in windows operation system, when running unit tests.
## How was this patch tested?
unit tests
Closes#23363 from yanlin-Lynn/unit-test-windows.
Authored-by: wangyanlin01 <wangyanlin01@baidu.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Variation of https://github.com/apache/spark/pull/20500
I cheated by not referencing fields or columns at all as this exception propagates in contexts where both would be applicable.
## How was this patch tested?
Existing tests
Closes#23373 from srowen/SPARK-14023.2.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
The PRs #23150 and #23196 switched JSON and CSV datasources on new formatter for dates/timestamps which is based on `DateTimeFormatter`. In this PR, I replaced `SimpleDateFormat` by `DateTimeFormatter` to reflect the changes.
Closes#23374 from MaxGekk/java-time-docs.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
GetStructField with different optional names should be semantically equal. We will use this as building block to compare the nested fields used in the plans to be optimized by catalyst optimizer.
This PR also fixes a bug below that accessing nested fields with different cases in case insensitive mode will result `AnalysisException`.
```
sql("create table t (s struct<i: Int>) using json")
sql("select s.I from t group by s.i")
```
which is currently failing
```
org.apache.spark.sql.AnalysisException: expression 'default.t.`s`' is neither present in the group by, nor is it an aggregate function
```
as cloud-fan pointed out.
## How was this patch tested?
New tests are added.
Closes#23353 from dbtsai/nestedEqual.
Lead-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: DB Tsai <dbtsai@dbtsai.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
This patch adds explanation of `why "spark.sql.shuffle.partitions" keeps unchanged in structured streaming`, which couple of users already wondered and some of them even thought it as a bug.
This patch would help other end users to know about such behavior before they find by theirselves and being wondered.
## How was this patch tested?
No need to test because this is a simple addition on guide doc with markdown editor.
Closes#22238 from HeartSaVioR/SPARK-25245.
Lead-authored-by: Jungtaek Lim <kabhwan@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Use of `ProcessingTime` class was deprecated in favor of `Trigger.ProcessingTime` in Spark 2.2. And, [SPARK-21464](https://issues.apache.org/jira/browse/SPARK-21464) minimized it at 2.2.1. Recently, it grows again in test suites. This PR aims to clean up newly introduced deprecation warnings for Spark 3.0.
## How was this patch tested?
Pass the Jenkins with existing tests and manually check the warnings.
Closes#23367 from dongjoon-hyun/SPARK-26428.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
A followup of https://github.com/apache/spark/pull/23178 , to keep binary compability by using abstract class.
## How was this patch tested?
Manual test. I created a simple app with Spark 2.4
```
object TryUDF {
def main(args: Array[String]): Unit = {
val spark = SparkSession.builder().appName("test").master("local[*]").getOrCreate()
import spark.implicits._
val f1 = udf((i: Int) => i + 1)
println(f1.deterministic)
spark.range(10).select(f1.asNonNullable().apply($"id")).show()
spark.stop()
}
}
```
When I run it with current master, it fails with
```
java.lang.IncompatibleClassChangeError: Found interface org.apache.spark.sql.expressions.UserDefinedFunction, but class was expected
```
When I run it with this PR, it works
Closes#23351 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently, even if I explicitly disable Hive support in SparkR session as below:
```r
sparkSession <- sparkR.session("local[4]", "SparkR", Sys.getenv("SPARK_HOME"),
enableHiveSupport = FALSE)
```
produces when the Hadoop version is not supported by our Hive fork:
```
java.lang.reflect.InvocationTargetException
...
Caused by: java.lang.IllegalArgumentException: Unrecognized Hadoop major version number: 3.1.1.3.1.0.0-78
at org.apache.hadoop.hive.shims.ShimLoader.getMajorVersion(ShimLoader.java:174)
at org.apache.hadoop.hive.shims.ShimLoader.loadShims(ShimLoader.java:139)
at org.apache.hadoop.hive.shims.ShimLoader.getHadoopShims(ShimLoader.java:100)
at org.apache.hadoop.hive.conf.HiveConf$ConfVars.<clinit>(HiveConf.java:368)
... 43 more
Error in handleErrors(returnStatus, conn) :
java.lang.ExceptionInInitializerError
at org.apache.hadoop.hive.conf.HiveConf.<clinit>(HiveConf.java:105)
at java.lang.Class.forName0(Native Method)
at java.lang.Class.forName(Class.java:348)
at org.apache.spark.util.Utils$.classForName(Utils.scala:193)
at org.apache.spark.sql.SparkSession$.hiveClassesArePresent(SparkSession.scala:1116)
at org.apache.spark.sql.api.r.SQLUtils$.getOrCreateSparkSession(SQLUtils.scala:52)
at org.apache.spark.sql.api.r.SQLUtils.getOrCreateSparkSession(SQLUtils.scala)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
```
The root cause is that:
```
SparkSession.hiveClassesArePresent
```
check if the class is loadable or not to check if that's in classpath but `org.apache.hadoop.hive.conf.HiveConf` has a check for Hadoop version as static logic which is executed right away. This throws an `IllegalArgumentException` and that's not caught:
36edbac1c8/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L1113-L1121)
So, currently, if users have a Hive built-in Spark with unsupported Hadoop version by our fork (namely 3+), there's no way to use SparkR even though it could work.
This PR just propose to change the order of bool comparison so that we can don't execute `SparkSession.hiveClassesArePresent` when:
1. `enableHiveSupport` is explicitly disabled
2. `spark.sql.catalogImplementation` is `in-memory`
so that we **only** check `SparkSession.hiveClassesArePresent` when Hive support is explicitly enabled by short circuiting.
## How was this patch tested?
It's difficult to write a test since we don't run tests against Hadoop 3 yet. See https://github.com/apache/spark/pull/21588. Manually tested.
Closes#23356 from HyukjinKwon/SPARK-26422.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
`HadoopFileWholeTextReader` and `HadoopFileLinesReader` will be eventually called in `FileSourceScanExec`.
In fact, locality has been implemented in `FileScanRDD`, even if we implement it in `HadoopFileWholeTextReader ` and `HadoopFileLinesReader`, it would be useless.
So I think these `TODO` can be removed.
## How was this patch tested?
N/A
Closes#23339 from 10110346/noneededtodo.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
`SQLConf` is supposed to be serializable. However, currently it is not serializable in `WithTestConf`. `WithTestConf` uses the method `overrideConfs` in closure, while the classes which implements it (`TestHiveSessionStateBuilder` and `TestSQLSessionStateBuilder`) are not serializable.
This PR is to use a local variable to fix it.
## How was this patch tested?
Add unit test.
Closes#23352 from gengliangwang/serializableSQLConf.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Currently, when we infer the schema for scala/java decimals, we return as data type the `SYSTEM_DEFAULT` implementation, ie. the decimal type with precision 38 and scale 18. But this is not right, as we know nothing about the right precision and scale and these values can be not enough to store the data. This problem arises in particular with UDF, where we cast all the input of type `DecimalType` to a `DecimalType(38, 18)`: in case this is not enough, null is returned as input for the UDF.
The PR defines a custom handling for casting to the expected data types for ScalaUDF: the decimal precision and scale is picked from the input, so no casting to different and maybe wrong percision and scale happens.
## How was this patch tested?
added UTs
Closes#23308 from mgaido91/SPARK-26308.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Deprecate Row.merge
## How was this patch tested?
N/A
Closes#23271 from KyleLi1985/master.
Authored-by: 李亮 <liang.li.work@outlook.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
In Spark 2.3.0 and previous versions, Hive CTAS command will convert to use data source to write data into the table when the table is convertible. This behavior is controlled by the configs like HiveUtils.CONVERT_METASTORE_ORC and HiveUtils.CONVERT_METASTORE_PARQUET.
In 2.3.1, we drop this optimization by mistake in the PR [SPARK-22977](https://github.com/apache/spark/pull/20521/files#r217254430). Since that Hive CTAS command only uses Hive Serde to write data.
This patch adds this optimization back to Hive CTAS command. This patch adds OptimizedCreateHiveTableAsSelectCommand which uses data source to write data.
## How was this patch tested?
Added test.
Closes#22514 from viirya/SPARK-25271-2.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
For better test coverage, this pr proposed to use the 4 mixed config sets of `WHOLESTAGE_CODEGEN_ENABLED` and `CODEGEN_FACTORY_MODE` when running `SQLQueryTestSuite`:
1. WHOLESTAGE_CODEGEN_ENABLED=true, CODEGEN_FACTORY_MODE=CODEGEN_ONLY
2. WHOLESTAGE_CODEGEN_ENABLED=false, CODEGEN_FACTORY_MODE=CODEGEN_ONLY
3. WHOLESTAGE_CODEGEN_ENABLED=true, CODEGEN_FACTORY_MODE=NO_CODEGEN
4. WHOLESTAGE_CODEGEN_ENABLED=false, CODEGEN_FACTORY_MODE=NO_CODEGEN
This pr also moved some existing tests into `ExplainSuite` because explain output results are different between codegen and interpreter modes.
## How was this patch tested?
Existing tests.
Closes#23213 from maropu/InterpreterModeTest.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a small clean up.
By design catalyst rules should be orthogonal: each rule should have its own responsibility. However, the `ColumnPruning` rule does not only do column pruning, but also remove no-op project and window.
This PR updates the `RemoveRedundantProject` rule to remove no-op window as well, and clean up the `ColumnPruning` rule to only do column pruning.
## How was this patch tested?
existing tests
Closes#23343 from cloud-fan/column-pruning.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
In `ReplaceExceptWithFilter` we do not consider properly the case in which the condition returns NULL. Indeed, in that case, since negating NULL still returns NULL, so it is not true the assumption that negating the condition returns all the rows which didn't satisfy it, rows returning NULL may not be returned. This happens when constraints inferred by `InferFiltersFromConstraints` are not enough, as it happens with `OR` conditions.
The rule had also problems with non-deterministic conditions: in such a scenario, this rule would change the probability of the output.
The PR fixes these problem by:
- returning False for the condition when it is Null (in this way we do return all the rows which didn't satisfy it);
- avoiding any transformation when the condition is non-deterministic.
## How was this patch tested?
added UTs
Closes#23315 from mgaido91/SPARK-26366.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Currently, SQL configs are not propagated to executors while schema inferring in CSV datasource. For example, changing of `spark.sql.legacy.timeParser.enabled` does not impact on inferring timestamp types. In the PR, I propose to fix the issue by wrapping schema inferring action using `SQLExecution.withSQLConfPropagated`.
## How was this patch tested?
Added logging to `TimestampFormatter`:
```patch
-object TimestampFormatter {
+object TimestampFormatter extends Logging {
def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = {
if (SQLConf.get.legacyTimeParserEnabled) {
+ logError("LegacyFallbackTimestampFormatter is being used")
new LegacyFallbackTimestampFormatter(format, timeZone, locale)
} else {
+ logError("Iso8601TimestampFormatter is being used")
new Iso8601TimestampFormatter(format, timeZone, locale)
}
}
```
and run the command in `spark-shell`:
```shell
$ ./bin/spark-shell --conf spark.sql.legacy.timeParser.enabled=true
```
```scala
scala> Seq("2010|10|10").toDF.repartition(1).write.mode("overwrite").text("/tmp/foo")
scala> spark.read.option("inferSchema", "true").option("header", "false").option("timestampFormat", "yyyy|MM|dd").csv("/tmp/foo").printSchema()
18/12/18 10:47:27 ERROR TimestampFormatter: LegacyFallbackTimestampFormatter is being used
root
|-- _c0: timestamp (nullable = true)
```
Closes#23345 from MaxGekk/csv-schema-infer-propagate-configs.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This PR proposes to use foreach instead of misuse of map (for Unit). This could cause some weird errors potentially and it's not a good practice anyway. See also SPARK-16694
## How was this patch tested?
N/A
Closes#23341 from HyukjinKwon/followup-SPARK-26081.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
The `JsonInferSchema` class is extended to support `TimestampType` inferring from string fields in JSON input:
- If the `prefersDecimal` option is set to `true`, it tries to infer decimal type from the string field.
- If decimal type inference fails or `prefersDecimal` is disabled, `JsonInferSchema` tries to infer `TimestampType`.
- If timestamp type inference fails, `StringType` is returned as the inferred type.
## How was this patch tested?
Added new test suite - `JsonInferSchemaSuite` to check date and timestamp types inferring from JSON using `JsonInferSchema` directly. A few tests were added `JsonSuite` to check type merging and roundtrip tests. This changes was tested by `JsonSuite`, `JsonExpressionsSuite` and `JsonFunctionsSuite` as well.
Closes#23201 from MaxGekk/json-infer-time.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes#22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
SinkProgress should report similar properties like SourceProgress as long as they are available for given Sink. Count of written rows is metric availble for all Sinks. Since relevant progress information is with respect to commited rows, ideal object to carry this info is WriterCommitMessage. For brevity the implementation will focus only on Sinks with API V2 and on Micro Batch mode. Implemention for Continuous mode will be provided at later date.
### Before
```
{"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317"}
```
### After
```
{"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317","numOutputRows":5000}
```
### This PR is related to:
- https://issues.apache.org/jira/browse/SPARK-24647
- https://issues.apache.org/jira/browse/SPARK-21313
## How was this patch tested?
Existing and new unit tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#21919 from vackosar/feature/SPARK-24933-numOutputRows.
Lead-authored-by: Vaclav Kosar <admin@vaclavkosar.com>
Co-authored-by: Kosar, Vaclav: Functions Transformation <Vaclav.Kosar@barclayscapital.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
1. rename `FormatterUtils` to `DateTimeFormatterHelper`, and move it to a separated file
2. move `DateFormatter` and its implementation to a separated file
3. mark some methods as private
4. add `override` to some methods
## How was this patch tested?
existing tests
Closes#23329 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This PR is a follow-up of the PR https://github.com/apache/spark/pull/17899. It is to add the rule TransposeWindow the optimizer batch.
## How was this patch tested?
The existing tests.
Closes#23222 from gatorsmile/followupSPARK-20636.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
- The original comment about `updateDriverMetrics` is not right.
- Refactor the code to ensure `selectedPartitions ` has been set before sending the driver-side metrics.
- Restore the original name, which is more general and extendable.
## How was this patch tested?
The existing tests.
Closes#23328 from gatorsmile/followupSpark-26142.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The optimizer rule `org.apache.spark.sql.catalyst.optimizer.ReorderJoin` performs join reordering on inner joins. This was introduced from SPARK-12032 (https://github.com/apache/spark/pull/10073) in 2015-12.
After it had reordered the joins, though, it didn't check whether or not the output attribute order is still the same as before. Thus, it's possible to have a mismatch between the reordered output attributes order vs the schema that a DataFrame thinks it has.
The same problem exists in the CBO version of join reordering (`CostBasedJoinReorder`) too.
This can be demonstrated with the example:
```scala
spark.sql("create table table_a (x int, y int) using parquet")
spark.sql("create table table_b (i int, j int) using parquet")
spark.sql("create table table_c (a int, b int) using parquet")
val df = spark.sql("""
with df1 as (select * from table_a cross join table_b)
select * from df1 join table_c on a = x and b = i
""")
```
here's what the DataFrame thinks:
```
scala> df.printSchema
root
|-- x: integer (nullable = true)
|-- y: integer (nullable = true)
|-- i: integer (nullable = true)
|-- j: integer (nullable = true)
|-- a: integer (nullable = true)
|-- b: integer (nullable = true)
```
here's what the optimized plan thinks, after join reordering:
```
scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}"))
|-- x: integer
|-- y: integer
|-- a: integer
|-- b: integer
|-- i: integer
|-- j: integer
```
If we exclude the `ReorderJoin` rule (using Spark 2.4's optimizer rule exclusion feature), it's back to normal:
```
scala> spark.conf.set("spark.sql.optimizer.excludedRules", "org.apache.spark.sql.catalyst.optimizer.ReorderJoin")
scala> val df = spark.sql("with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i")
df: org.apache.spark.sql.DataFrame = [x: int, y: int ... 4 more fields]
scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}"))
|-- x: integer
|-- y: integer
|-- i: integer
|-- j: integer
|-- a: integer
|-- b: integer
```
Note that this output attribute ordering problem leads to data corruption, and can manifest itself in various symptoms:
* Silently corrupting data, if the reordered columns happen to either have matching types or have sufficiently-compatible types (e.g. all fixed length primitive types are considered as "sufficiently compatible" in an `UnsafeRow`), then only the resulting data is going to be wrong but it might not trigger any alarms immediately. Or
* Weird Java-level exceptions like `java.lang.NegativeArraySizeException`, or even SIGSEGVs.
## How was this patch tested?
Added new unit test in `JoinReorderSuite` and new end-to-end test in `JoinSuite`.
Also made `JoinReorderSuite` and `StarJoinReorderSuite` assert more strongly on maintaining output attribute order.
Closes#23303 from rednaxelafx/fix-join-reorder.
Authored-by: Kris Mok <rednaxelafx@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The `CSVInferSchema` class is extended to support inferring of `DateType` from CSV input. The attempt to infer `DateType` is performed after inferring `TimestampType`.
## How was this patch tested?
Added new test for inferring date types from CSV . It was also tested by existing suites like `CSVInferSchemaSuite`, `CsvExpressionsSuite`, `CsvFunctionsSuite` and `CsvSuite`.
Closes#23202 from MaxGekk/csv-date-inferring.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
CSV parsing accidentally uses the previous good value for a bad input field. See example in Jira.
This PR ensures that the associated column is set to null when an input field cannot be converted.
## How was this patch tested?
Added new test.
Ran all SQL unit tests (testOnly org.apache.spark.sql.*).
Ran pyspark tests for pyspark-sql
Closes#23323 from bersprockets/csv-bad-field.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When there is a self-join as result of a IN subquery, the join condition may be invalid, resulting in trivially true predicates and return wrong results.
The PR deduplicates the subquery output in order to avoid the issue.
## How was this patch tested?
added UT
Closes#23057 from mgaido91/SPARK-26078.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to switch on **java.time API** for parsing timestamps and dates from JSON inputs with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behavior with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates.
## How was this patch tested?
It was tested by `JsonExpressionsSuite`, `JsonFunctionsSuite` and `JsonSuite`.
Closes#23196 from MaxGekk/json-time-parser.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Multiple SparkContexts are discouraged and it has been warning for last 4 years, see SPARK-4180. It could cause arbitrary and mysterious error cases, see SPARK-2243.
Honestly, I didn't even know Spark still allows it, which looks never officially supported, see SPARK-2243.
I believe It should be good timing now to remove this configuration.
## How was this patch tested?
Each doc was manually checked and manually tested:
```
$ ./bin/spark-shell --conf=spark.driver.allowMultipleContexts=true
...
scala> new SparkContext()
org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:939)
...
org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2435)
at scala.Option.foreach(Option.scala:274)
at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2432)
at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2509)
at org.apache.spark.SparkContext.<init>(SparkContext.scala:80)
at org.apache.spark.SparkContext.<init>(SparkContext.scala:112)
... 49 elided
```
Closes#23311 from HyukjinKwon/SPARK-26362.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When using ordinals to access linked list, the time cost is O(n).
## How was this patch tested?
Existing tests.
Closes#23280 from CarolinePeng/update_Two.
Authored-by: CarolinPeng <00244106@zte.intra>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
When using a higher-order function with the same variable name as the existing columns in `Filter` or something which uses `Analyzer.resolveExpressionBottomUp` during the resolution, e.g.,:
```scala
val df = Seq(
(Seq(1, 9, 8, 7), 1, 2),
(Seq(5, 9, 7), 2, 2),
(Seq.empty, 3, 2),
(null, 4, 2)
).toDF("i", "x", "d")
checkAnswer(df.filter("exists(i, x -> x % d == 0)"),
Seq(Row(Seq(1, 9, 8, 7), 1, 2)))
checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"),
Seq(Row(1)))
```
the following exception happens:
```
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.BoundReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
at scala.collection.TraversableLike.map(TraversableLike.scala:237)
at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
at scala.collection.AbstractTraversable.map(Traversable.scala:108)
at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.$anonfun$functionsForEval$1(higherOrderFunctions.scala:147)
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
at scala.collection.immutable.List.foreach(List.scala:392)
at scala.collection.TraversableLike.map(TraversableLike.scala:237)
at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
at scala.collection.immutable.List.map(List.scala:298)
at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval(higherOrderFunctions.scala:145)
at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval$(higherOrderFunctions.scala:145)
at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval$lzycompute(higherOrderFunctions.scala:369)
at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval(higherOrderFunctions.scala:369)
at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval(higherOrderFunctions.scala:176)
at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval$(higherOrderFunctions.scala:176)
at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionForEval(higherOrderFunctions.scala:369)
at org.apache.spark.sql.catalyst.expressions.ArrayExists.nullSafeEval(higherOrderFunctions.scala:387)
at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval(higherOrderFunctions.scala:190)
at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval$(higherOrderFunctions.scala:185)
at org.apache.spark.sql.catalyst.expressions.ArrayExists.eval(higherOrderFunctions.scala:369)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3(basicPhysicalOperators.scala:216)
at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3$adapted(basicPhysicalOperators.scala:215)
...
```
because the `UnresolvedAttribute`s in `LambdaFunction` are unexpectedly resolved by the rule.
This pr modified to use a placeholder `UnresolvedNamedLambdaVariable` to prevent unexpected resolution.
## How was this patch tested?
Added a test and modified some tests.
Closes#23320 from ueshin/issues/SPARK-26370/hof_resolution.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
I was looking at the code and it was a bit difficult to see the life cycle of InMemoryFileIndex passed into getOrInferFileFormatSchema, because once it is passed in, and another time it was created in getOrInferFileFormatSchema. It'd be easier to understand the life cycle if we move the creation of it out.
## How was this patch tested?
This is a simple code move and should be covered by existing tests.
Closes#23317 from rxin/SPARK-26368.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Regarding the performance issue of SPARK-26155, it reports the issue on TPC-DS. I think it is better to add a benchmark for `LongToUnsafeRowMap` which is the root cause of performance regression.
It can be easier to show performance difference between different metric implementations in `LongToUnsafeRowMap`.
## How was this patch tested?
Manually run added benchmark.
Closes#23284 from viirya/SPARK-26337.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Added query status updates to ContinuousExecution.
## How was this patch tested?
Existing unit tests + added ContinuousQueryStatusAndProgressSuite.
Closes#23095 from gaborgsomogyi/SPARK-23886.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
As discussed in https://github.com/apache/spark/pull/23208/files#r239684490 , we should put `newScanBuilder` in read related mix-in traits like `SupportsBatchRead`, to support write-only table.
In the `Append` operator, we should skip schema validation if not necessary. In the future we would introduce a capability API, so that data source can tell Spark that it doesn't want to do validation.
## How was this patch tested?
existing tests.
Closes#23266 from cloud-fan/ds-read.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
There are some comments issues left when `ConvertToLocalRelation` rule was added (see #22205/[SPARK-25212](https://issues.apache.org/jira/browse/SPARK-25212)). This PR fixes those comments issues.
## How was this patch tested?
N/A
Closes#23273 from seancxmao/ConvertToLocalRelation-doc.
Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
cleanup some tests to make sure expression is resolved during test.
## How was this patch tested?
test-only PR
Closes#23297 from cloud-fan/test.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Some documents of `Distribution/Partitioning` are stale and misleading, this PR fixes them:
1. `Distribution` never have intra-partition requirement
2. `OrderedDistribution` does not require tuples that share the same value being colocated in the same partition.
3. `RangePartitioning` can provide a weaker guarantee for a prefix of its `ordering` expressions.
## How was this patch tested?
comment-only PR.
Closes#23249 from cloud-fan/doc.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Follow up pr for #23207, include following changes:
- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.
## How was this patch tested?
Existing tests.
Closes#23286 from xuanyuanking/SPARK-26193-follow.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
If `checkForContinuous` is called ( `checkForStreaming` is called in `checkForContinuous` ), the `checkForStreaming` mothod will be called twice in `createQuery` , this is not necessary, and the `checkForStreaming` method has a lot of statements, so it's better to remove one of them.
## How was this patch tested?
Existing unit tests in `StreamingQueryManagerSuite` and `ContinuousAggregationSuite`
Closes#23251 from 10110346/isUnsupportedOperationCheckEnabled.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
In the PR, I propose to return partial results from JSON datasource and JSON functions in the PERMISSIVE mode if some of JSON fields are parsed and converted to desired types successfully. The changes are made only for `StructType`. Whole bad JSON records are placed into the corrupt column specified by the `columnNameOfCorruptRecord` option or SQL config.
Partial results are not returned for malformed JSON input.
## How was this patch tested?
Added new UT which checks converting JSON strings with one invalid and one valid field at the end of the string.
Closes#23253 from MaxGekk/json-bad-record.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is a regression introduced by https://github.com/apache/spark/pull/22104 at Spark 2.4.0.
When we have Python UDF in subquery, we will hit an exception
```
Caused by: java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.AttributeReference cannot be cast to org.apache.spark.sql.catalyst.expressions.PythonUDF
at scala.collection.immutable.Stream.map(Stream.scala:414)
at org.apache.spark.sql.execution.python.EvalPythonExec.$anonfun$doExecute$2(EvalPythonExec.scala:98)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:815)
...
```
https://github.com/apache/spark/pull/22104 turned `ExtractPythonUDFs` from a physical rule to optimizer rule. However, there is a difference between a physical rule and optimizer rule. A physical rule always runs once, an optimizer rule may be applied twice on a query tree even the rule is located in a batch that only runs once.
For a subquery, the `OptimizeSubqueries` rule will execute the entire optimizer on the query plan inside subquery. Later on subquery will be turned to joins, and the optimizer rules will be applied to it again.
Unfortunately, the `ExtractPythonUDFs` rule is not idempotent. When it's applied twice on a query plan inside subquery, it will produce a malformed plan. It extracts Python UDF from Python exec plans.
This PR proposes 2 changes to be double safe:
1. `ExtractPythonUDFs` should skip python exec plans, to make the rule idempotent
2. `ExtractPythonUDFs` should skip subquery
## How was this patch tested?
a new test.
Closes#23248 from cloud-fan/python.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
`RDDConversions` would get disproportionately slower as the number of columns in the query increased,
for the type of `converters` before is `scala.collection.immutable.::` which is a subtype of list.
This PR removing `RDDConversions` and using `RowEncoder` to convert the Row to InternalRow.
The test of `PrunedScanSuite` for 2000 columns and 20k rows takes 409 seconds before this PR, and 361 seconds after.
## How was this patch tested?
Test case of `PrunedScanSuite`
Closes#23262 from eatoncys/toarray.
Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
… incorrect.
## What changes were proposed in this pull request?
In the reported heartbeat information, the unit of the memory data is bytes, which is converted by the formatBytes() function in the utils.js file before being displayed in the interface. The cardinality of the unit conversion in the formatBytes function is 1000, which should be 1024.
Change the cardinality of the unit conversion in the formatBytes function to 1024.
## How was this patch tested?
manual tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#22683 from httfighter/SPARK-25696.
Lead-authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Co-authored-by: han.tiantian@zte.com.cn <han.tiantian@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Add appendReadColumns Unit Test for HiveShimSuite.
## How was this patch tested?
```
$ build/sbt
> project hive
> testOnly *HiveShimSuite
```
Closes#23268 from sadhen/refactor/hiveshim.
Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is a Spark 2.3 regression introduced in https://github.com/apache/spark/pull/20521. We should add the partition info for InsertIntoHiveTable in CreateHiveTableAsSelectCommand. Otherwise, we will hit the following error by running the newly added test case:
```
[info] - CTAS: INSERT a partitioned table using Hive serde *** FAILED *** (829 milliseconds)
[info] org.apache.spark.SparkException: Requested partitioning does not match the tab1 table:
[info] Requested partitions:
[info] Table partitions: part
[info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.processInsert(InsertIntoHiveTable.scala:179)
[info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:107)
```
## How was this patch tested?
Added a test case.
Closes#23255 from gatorsmile/fixCTAS.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`.
2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency.
3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.
## How was this patch tested?
Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.
Closes#23207 from xuanyuanking/SPARK-26193.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
A followup of https://github.com/apache/spark/pull/23043
There are 4 places we need to deal with NaN and -0.0:
1. comparison expressions. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
2. Join keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
3. grouping keys. `-0.0` and `0.0` should be assigned to the same group. Different NaNs should be assigned to the same group.
4. window partition keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
The case 1 is OK. Our comparison already handles NaN and -0.0, and for struct/array/map, we will recursively compare the fields/elements.
Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different NaNs have different binary representation, and the same thing happens for -0.0 and 0.0.
To fix it, a simple solution is: normalize float/double when building unsafe data (`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`). Then we don't need to worry about it anymore.
Following this direction, this PR moves the handling of NaN and -0.0 from `Platform` to `UnsafeWriter`, so that places like `UnsafeRow.setFloat` will not handle them, which reduces the perf overhead. It's also easier to add comments explaining why we do it in `UnsafeWriter`.
## How was this patch tested?
existing tests
Closes#23239 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Parquet files appear to have nullability info when being written, not being read.
## How was this patch tested?
Some test code: (running spark 2.3, but the relevant code in DataSource looks identical on master)
case class NullTest(bo: Boolean, opbol: Option[Boolean])
val testDf = spark.createDataFrame(Seq(NullTest(true, Some(false))))
defined class NullTest
testDf: org.apache.spark.sql.DataFrame = [bo: boolean, opbol: boolean]
testDf.write.parquet("s3://asana-stats/tmp_dima/parquet_check_schema")
spark.read.parquet("s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet4").printSchema()
root
|-- bo: boolean (nullable = true)
|-- opbol: boolean (nullable = true)
Meanwhile, the parquet file formed does have nullable info:
[]batchprod-report000:/tmp/dimakamalov-batch$ aws s3 ls s3://asana-stats/tmp_dima/parquet_check_schema/
2018-10-17 21:03:52 0 _SUCCESS
2018-10-17 21:03:50 504 part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batchprod-report000:/tmp/dimakamalov-batch$ aws s3 cp s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet .
download: s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet to ./part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batchprod-report000:/tmp/dimakamalov-batch$ java -jar parquet-tools-1.8.2.jar schema part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
message spark_schema {
required boolean bo;
optional boolean opbol;
}
Closes#22759 from dima-asana/dima-asana-nullable-parquet-doc.
Authored-by: dima-asana <42555784+dima-asana@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This is a follow-up of #23031 to rename the config name to `spark.sql.legacy.setCommandRejectsSparkCoreConfs`.
## How was this patch tested?
Existing tests.
Closes#23245 from ueshin/issues/SPARK-26060/rename_config.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Currently if user provides data schema, partition column values are converted as per it. But if the conversion failed, e.g. converting string to int, the column value is null.
This PR proposes to throw exception in such case, instead of converting into null value silently:
1. These null partition column values doesn't make sense to users in most cases. It is better to show the conversion failure, and then users can adjust the schema or ETL jobs to fix it.
2. There are always exceptions on such conversion failure for non-partition data columns. Partition columns should have the same behavior.
We can reproduce the case above as following:
```
/tmp/testDir
├── p=bar
└── p=foo
```
If we run:
```
val schema = StructType(Seq(StructField("p", IntegerType, false)))
spark.read.schema(schema).csv("/tmp/testDir/").show()
```
We will get:
```
+----+
| p|
+----+
|null|
|null|
+----+
```
## How was this patch tested?
Unit test
Closes#23215 from gengliangwang/SPARK-26263.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
`enablePerfMetrics `was originally designed in `BytesToBytesMap `to control `getNumHashCollisions getTimeSpentResizingNs getAverageProbesPerLookup`.
However, as the Spark version gradual progress. this parameter is only used for `getAverageProbesPerLookup ` and always given to true when using `BytesToBytesMap`.
it is also dangerous to determine whether `getAverageProbesPerLookup `opens and throws an `IllegalStateException `exception.
So this pr will be remove `enablePerfMetrics `parameter from `BytesToBytesMap`. thanks.
## How was this patch tested?
the existed test cases.
Closes#23244 from heary-cao/enablePerfMetrics.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
When executing `toPandas` with Arrow enabled, partitions that arrive in the JVM out-of-order must be buffered before they can be send to Python. This causes an excess of memory to be used in the driver JVM and increases the time it takes to complete because data must sit in the JVM waiting for preceding partitions to come in.
This change sends un-ordered partitions to Python as soon as they arrive in the JVM, followed by a list of partition indices so that Python can assemble the data in the correct order. This way, data is not buffered at the JVM and there is no waiting on particular partitions so performance will be increased.
Followup to #21546
## How was this patch tested?
Added new test with a large number of batches per partition, and test that forces a small delay in the first partition. These test that partitions are collected out-of-order and then are are put in the correct order in Python.
## Performance Tests - toPandas
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.
Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
start = time.time()
_ = df.toPandas()
elapsed = time.time() - start
```
Spark config
```
spark.driver.memory 5g
spark.executor.memory 5g
spark.driver.maxResultSize 2g
spark.sql.execution.arrow.enabled true
```
Current Master w/ Arrow stream | This PR
---------------------|------------
5.16207 | 4.342533
5.133671 | 4.399408
5.147513 | 4.468471
5.105243 | 4.36524
5.018685 | 4.373791
Avg Master | Avg This PR
------------------|--------------
5.1134364 | 4.3898886
Speedup of **1.164821449**
Closes#22275 from BryanCutler/arrow-toPandas-oo-batches-SPARK-25274.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
## What changes were proposed in this pull request?
this code come from PR: https://github.com/apache/spark/pull/11190,
but this code has never been used, only since PR: https://github.com/apache/spark/pull/14548,
Let's continue fix it. thanks.
## How was this patch tested?
N / A
Closes#23227 from heary-cao/unuseSparkPlan.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The PR starts from the [comment](https://github.com/apache/spark/pull/23124#discussion_r236112390) in the main one and it aims at:
- simplifying the code for `MapConcat`;
- be more precise in checking the limit size.
## How was this patch tested?
existing tests
Closes#23217 from mgaido91/SPARK-25829_followup.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
When we encode a Decimal from external source we don't check for overflow. That method is useful not only in order to enforce that we can represent the correct value in the specified range, but it also changes the underlying data to the right precision/scale. Since in our code generation we assume that a decimal has exactly the same precision and scale of its data type, missing to enforce it can lead to corrupted output/results when there are subsequent transformations.
## How was this patch tested?
added UT
Closes#23210 from mgaido91/SPARK-26233.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
In the PR, I propose to use **java.time API** for parsing timestamps and dates from CSV content with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behaviour with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates.
## How was this patch tested?
It was tested by `UnivocityParserSuite`, `CsvExpressionsSuite`, `CsvFunctionsSuite` and `CsvSuite`.
Closes#23150 from MaxGekk/time-parser.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
![image](https://user-images.githubusercontent.com/5399861/49172173-42ad9800-f37b-11e8-8135-7adc323357ae.png)
It will throw:
```
requirement failed: When reading JDBC data sources, users need to specify all or none for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and 'numPartitions'
```
and
```
User-defined partition column subq.c1 not found in the JDBC relation ...
```
This PR fix this error example.
## How was this patch tested?
manual tests
Closes#23170 from wangyum/SPARK-24499.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the DDLSuit, there are four test cases have the same codes , writing a function can combine the same code.
## How was this patch tested?
existing tests.
Closes#23194 from CarolinePeng/Update_temp.
Authored-by: 彭灿00244106 <00244106@zte.intra>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
## What changes were proposed in this pull request?
In SPARK-23711, we have implemented the expression fallback logic to an interpreted mode. So, this pr fixed code to support the same fallback mode in `SafeProjection` based on `CodeGeneratorWithInterpretedFallback`.
## How was this patch tested?
Add tests in `CodeGeneratorWithInterpretedFallbackSuite` and `UnsafeRowConverterSuite`.
Closes#22468 from maropu/SPARK-25374-3.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently in the Analyzer, we have two methods 1) Resolve 2)ResolveExpressions that are called at different code paths to resolve attributes, column ordinal and extract value expressions. ~~In this PR, we combine the two into one method to make sure, there is only one method that is tasked with resolving the attributes.~~
Update the description of the methods and use better names to make it easier to know when to make use of one method vs the other.
## How was this patch tested?
Existing tests.
Closes#22899 from dilipbiswal/SPARK-25573-final.
Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This pr is to return an empty config set when regenerating the golden files in `SQLQueryTestSuite`.
This is the follow-up of #22512.
## How was this patch tested?
N/A
Closes#23212 from maropu/SPARK-25498-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Since `AggregationIterator` uses `MutableProjection` for `UnsafeRow`, `InterpretedMutableProjection` needs to handle `UnsafeRow` as buffer internally for fixed-length types only.
## How was this patch tested?
Run 'SQLQueryTestSuite' with the interpreted mode.
Closes#22512 from maropu/InterpreterTest.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
For now the `hasMinMaxStats` will return the same as `hasCountStats`, which is obviously not as expected.
## How was this patch tested?
Existing tests.
Closes#23152 from adrian-wang/minmaxstats.
Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Partition column name is required to be unique under the same directory. The following paths are invalid partitioned directory:
```
hdfs://host:9000/path/a=1
hdfs://host:9000/path/b=2
```
If case sensitive, the following paths should be invalid too:
```
hdfs://host:9000/path/a=1
hdfs://host:9000/path/A=2
```
Since column 'a' and 'A' are different, and it is wrong to use either one as the column name in partition schema.
Also, there is a `TODO` comment in the code. Currently the Spark doesn't validate such case when `CASE_SENSITIVE` enabled.
This PR is to resolve the problem.
## How was this patch tested?
Add unit test
Closes#23186 from gengliangwang/SPARK-26230.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to change behaviour of `UnivocityParser` and `FailureSafeParser`, and return all fields that were parsed and converted to expected types successfully instead of just returning a row with all `null`s for a bad input in the `PERMISSIVE` mode. For example, for CSV line `0,2013-111-11 12:13:14` and DDL schema `a int, b timestamp`, new result is `Row(0, null)`.
## How was this patch tested?
It was checked by existing tests from `CsvSuite` and `CsvFunctionsSuite`.
Closes#23120 from MaxGekk/failuresafe-partial-result.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
When build hash Map with one row of data and run out of memory, we should throw a SparkOutOfMemoryError exception, which is more accurate than SparkException. this PR fix it.
## How was this patch tested?
N / A
Closes#23190 from heary-cao/throwUnsafeHashedRelation.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
How to reproduce this issue:
```scala
scala> val meta = new org.apache.spark.sql.types.MetadataBuilder().putNull("key").build().json
java.lang.NullPointerException
at org.apache.spark.sql.types.Metadata$.org$apache$spark$sql$types$Metadata$$toJsonValue(Metadata.scala:196)
at org.apache.spark.sql.types.Metadata$$anonfun$1.apply(Metadata.scala:180)
```
This pr fix `NullPointerException` when `Metadata` serialize `null` values.
## How was this patch tested?
unit tests
Closes#23164 from wangyum/SPARK-26198.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Add headers to empty csv files when header=true, because otherwise these files are invalid when reading.
## How was this patch tested?
Added test for roundtrip of empty dataframe to csv file with headers and back in CSVSuite
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23173 from koertkuipers/feat-empty-csv-with-header.
Authored-by: Koert Kuipers <koert@tresata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
UnsupportedOperationException messages are not the same with method name.This PR correct these messages.
## How was this patch tested?
NA
Closes#23154 from lcqzte10192193/wid-lcq-1127.
Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
It's a bad idea to use case class as public API, as it has a very wide surface. For example, the `copy` method, its fields, the companion object, etc.
For a particular case, `UserDefinedFunction`. It has a private constructor, and I believe we only want users to access a few methods:`apply`, `nullable`, `asNonNullable`, etc.
However, all its fields, and `copy` method, and the companion object are public unexpectedly. As a result, we made many tricks to work around the binary compatibility issues.
This PR proposes to only make interfaces public, and hide implementations behind with a private class. Now `UserDefinedFunction` is a pure trait, and the concrete implementation is `SparkUserDefinedFunction`, which is private.
Changing class to interface is not binary compatible(but source compatible), so 3.0 is a good chance to do it.
This is the first PR to go with this direction. If it's accepted, I'll create a umbrella JIRA and fix all the public case classes.
## How was this patch tested?
existing tests.
Closes#23178 from cloud-fan/udf.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose filtering out all empty files inside of `FileSourceScanExec` and exclude them from file splits. It should reduce overhead of opening and reading files without any data, and as consequence datasources will not produce empty partitions for such files.
## How was this patch tested?
Added a test which creates an empty and non-empty files. If empty files are ignored in load, Text datasource in the `wholetext` mode must create only one partition for non-empty file.
Closes#23130 from MaxGekk/ignore-empty-files.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a follow pr of #23176.
`In` and `InSet` are semantically equal, so the tests for `In` should pass with `InSet`, and vice versa.
This combines those test cases.
## How was this patch tested?
The combined tests and existing tests.
Closes#23187 from ueshin/issues/SPARK-26211/in_inset_tests.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a small change for better debugging: to pass query uuid in IncrementalExecution, when we look at the QueryExecution in isolation to trace back the query.
## How was this patch tested?
N/A - just add some field for better debugging.
Closes#23192 from rxin/SPARK-26241.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
In an earlier PR, we missed measuring the optimization phase time for streaming queries. This patch adds it.
## How was this patch tested?
Given this is a debugging feature, and it is very convoluted to add tests to verify the phase is set properly, I am not introducing a streaming specific test.
Closes#23193 from rxin/SPARK-26226-1.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Correct some document description errors.
## How was this patch tested?
N/A
Closes#23162 from 10110346/docerror.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore to clean up SparkCore test cases. thanks.
## How was this patch tested?
N / A
Closes#23151 from heary-cao/withCreateTempDir.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This patch changes the query plan tracker added earlier to report phase timeline, rather than just a duration for each phase. This way, we can easily find time that's unaccounted for.
## How was this patch tested?
Updated test cases to reflect that.
Closes#23183 from rxin/SPARK-26226.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This is the first step of the data source v2 API refactor [proposal](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)
It adds the new API for batch read, without removing the old APIs, as they are still needed for streaming sources.
More concretely, it adds
1. `TableProvider`, works like an anonymous catalog
2. `Table`, represents a structured data set.
3. `ScanBuilder` and `Scan`, a logical represents of data source scan
4. `Batch`, a physical representation of data source batch scan.
## How was this patch tested?
existing tests
Closes#23086 from cloud-fan/refactor-batch.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
This PR is to fix a regression introduced in: https://github.com/apache/spark/pull/21004/files#r236998030
If user specifies schema, Spark don't need to infer data type for of partition columns, otherwise the data type might not match with the one user provided.
E.g. for partition directory `p=4d`, after data type inference the column value will be `4.0`.
See https://issues.apache.org/jira/browse/SPARK-26188 for more details.
Note that user specified schema **might not cover all the data columns**:
```
val schema = new StructType()
.add("id", StringType)
.add("ex", ArrayType(StringType))
val df = spark.read
.schema(schema)
.format("parquet")
.load(src.toString)
assert(df.schema.toList === List(
StructField("ex", ArrayType(StringType)),
StructField("part", IntegerType), // inferred partitionColumn dataType
StructField("id", StringType))) // used user provided partitionColumn dataType
```
For the missing columns in user specified schema, Spark still need to infer their data types if `partitionColumnTypeInferenceEnabled` is enabled.
To implement the partially inference, refactor `PartitioningUtils.parsePartitions` and pass the user specified schema as parameter to cast partition values.
## How was this patch tested?
Add unit test.
Closes#23165 from gengliangwang/fixFileIndex.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. We should track `SparkConf` entries and make the command reject for such entries.
## How was this patch tested?
Added a test and existing tests.
Closes#23031 from ueshin/issues/SPARK-26060/set_command.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose using of the locale option to parse decimals from CSV input. After the changes, `UnivocityParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`.
## How was this patch tested?
Added a test for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales.
Closes#22979 from MaxGekk/decimal-parsing-locale.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Follow up for https://github.com/apache/spark/pull/23128, move sql read metrics relatives to `SQLShuffleMetricsReporter`, in order to put sql shuffle read metrics relatives closer and avoid possible problem about forgetting update SQLShuffleMetricsReporter while new metrics added by others.
## How was this patch tested?
Existing tests.
Closes#23175 from xuanyuanking/SPARK-26142-follow.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to postpone creation of `OutputStream`/`Univocity`/`JacksonGenerator` till the first row should be written. This prevents creation of empty files for empty partitions. So, no need to open and to read such files back while loading data from the location.
## How was this patch tested?
Added tests for Text, JSON and CSV datasource where empty dataset is written but should not produce any files.
Closes#23052 from MaxGekk/text-empty-files.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Currently `InSet` doesn't work properly for binary type, or struct and array type with null value in the set.
Because, as for binary type, the `HashSet` doesn't work properly for `Array[Byte]`, and as for struct and array type with null value in the set, the `ordering` will throw a `NPE`.
## How was this patch tested?
Added a few tests.
Closes#23176 from ueshin/issues/SPARK-26211/inset.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose using of the locale option to parse (and infer) decimals from JSON input. After the changes, `JacksonParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`. New behaviour can be switched off via SQL config `spark.sql.legacy.decimalParsing.enabled`.
## How was this patch tested?
Added 2 tests to `JsonExpressionsSuite` for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales:
- Inferring decimal type using locale from JSON field values
- Converting JSON field values to specified decimal type using the locales.
Closes#23132 from MaxGekk/json-decimal-parsing-locale.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.
This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.
updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.
For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.
## How was this patch tested?
updated tests and new tests
Closes#23124 from cloud-fan/map.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable.
This PR fixes this mistake.
## How was this patch tested?
a new test
Closes#23153 from cloud-fan/join.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
a followup of https://github.com/apache/spark/pull/23043 . Add a test to show the minor behavior change introduced by #23043 , and add migration guide.
## How was this patch tested?
a new test
Closes#23141 from cloud-fan/follow.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Implement codegen for `LocalTableScanExec` and `ExistingRDDExec`. Refactor to share code between `LocalTableScanExec`, `ExistingRDDExec`, `InputAdapter` and `RowDataSourceScanExec`.
The difference in `doProduce` between these four was that `ExistingRDDExec` and `RowDataSourceScanExec` triggered adding an `UnsafeProjection`, while `InputAdapter` and `LocalTableScanExec` did not.
In the new trait `InputRDDCodegen` I added a flag `createUnsafeProjection` which the operators set accordingly.
Note: `LocalTableScanExec` explicitly creates its input as `UnsafeRows`, so it was obvious why it doesn't need an `UnsafeProjection`. But if an `InputAdapter` may take input that is `InternalRows` but not `UnsafeRows`, then I think it doesn't need an unsafe projection just because any other operator that is its parent would do that. That assumes that that any parent operator would always result in some `UnsafeProjection` being eventually added, and hence the output of the `WholeStageCodegen` unit would be `UnsafeRows`. If these assumptions hold, I think `createUnsafeProjection` could be set to `(parent == null)`.
Note: Do not codegen `LocalTableScanExec` when it's the only operator. `LocalTableScanExec` has optimized driver-only `executeCollect` and `executeTake` code paths that are used to return `Command` results without starting Spark Jobs. They can no longer be used if the `LocalTableScanExec` gets optimized.
## How was this patch tested?
Covered and used in existing tests.
Closes#23127 from juliuszsompolski/SPARK-26159.
Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The purpose of this PR is supplement new test cases for a + 1,a + b and Rand in SetOperationSuite.
It comes from the comment of closed PR:#20541, thanks.
## How was this patch tested?
add new test cases
Closes#23138 from heary-cao/UnionPushTestCases.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Based on https://github.com/apache/spark/pull/22857 and https://github.com/apache/spark/pull/23079, this PR did a few updates
- Limit the data types of NULL to Boolean.
- Limit the input data type of replaceNullWithFalse to Boolean; throw an exception in the testing mode.
- Create a new file for the rule ReplaceNullWithFalseInPredicate
- Update the description of this rule.
## How was this patch tested?
Added a test case
Closes#23139 from gatorsmile/followupSpark-25860.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
This is follow-up of #21732. This patch inlines `isOptionType` method.
## How was this patch tested?
Existing tests.
Closes#23143 from viirya/SPARK-24762-followup.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR is to improve the code comments to document some common traits and traps about the expression.
## How was this patch tested?
N/A
Closes#23135 from gatorsmile/addcomments.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Create a new suite DataFrameSetOperationsSuite for the test cases of DataFrame/Dataset's set operations.
Also, add test cases of NULL handling for Array Except and Array Intersect.
## How was this patch tested?
N/A
Closes#23137 from gatorsmile/setOpsTest.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
SparkSQL doesn't support to encode `Option[Product]` as a top-level row now, because in SparkSQL entire top-level row can't be null.
However for use cases like Aggregator, it is reasonable to use `Option[Product]` as buffer and output column types. Due to above limitation, we don't do it for now.
This patch proposes to encode `Option[Product]` at top-level as single struct column. So we can work around the issue that entire top-level row can't be null.
To summarize encoding of `Product` and `Option[Product]`.
For `Product`, 1. at root level, the schema is all fields are flatten it into multiple columns. The `Product ` can't be null, otherwise it throws an exception.
```scala
val df = Seq((1 -> "a"), (2 -> "b")).toDF()
df.printSchema()
root
|-- _1: integer (nullable = false)
|-- _2: string (nullable = true)
```
2. At non-root level, `Product` is a struct type column.
```scala
val df = Seq((1, (1 -> "a")), (2, (2 -> "b")), (3, null)).toDF()
df.printSchema()
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: string (nullable = true)
```
For `Option[Product]`, 1. it was not supported at root level. After this change, it is a struct type column.
```scala
val df = Seq(Some(1 -> "a"), Some(2 -> "b"), None).toDF()
df.printSchema
root
|-- value: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: string (nullable = true)
```
2. At non-root level, it is also a struct type column.
```scala
val df = Seq((1, Some(1 -> "a")), (2, Some(2 -> "b")), (3, None)).toDF()
df.printSchema
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: string (nullable = true)
```
3. For use case like Aggregator, it was not supported too. After this change, we support to use `Option[Product]` as buffer/output column type.
```scala
val df = Seq(
OptionBooleanIntData("bob", Some((true, 1))),
OptionBooleanIntData("bob", Some((false, 2))),
OptionBooleanIntData("bob", None)).toDF()
val group = df
.groupBy("name")
.agg(OptionBooleanIntAggregator("isGood").toColumn.alias("isGood"))
group.printSchema
root
|-- name: string (nullable = true)
|-- isGood: struct (nullable = true)
| |-- _1: boolean (nullable = false)
| |-- _2: integer (nullable = false)
```
The buffer and output type of `OptionBooleanIntAggregator` is both `Option[(Boolean, Int)`.
## How was this patch tested?
Added test.
Closes#21732 from viirya/SPARK-24762.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR is to add back `unionAll`, which is widely used. The name is also consistent with our ANSI SQL. We also have the corresponding `intersectAll` and `exceptAll`, which were introduced in Spark 2.4.
## How was this patch tested?
Added a test case in DataFrameSuite
Closes#23131 from gatorsmile/addBackUnionAll.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The DOI foundation recommends [this new resolver](https://www.doi.org/doi_handbook/3_Resolution.html#3.8). Accordingly, this PR re`sed`s all static DOI links ;-)
## How was this patch tested?
It wasn't, since it seems as safe as a "[typo fix](https://spark.apache.org/contributing.html)".
In case any of the files is included from other projects, and should be updated there, please let me know.
Closes#23129 from katrinleinweber/resolve-DOIs-securely.
Authored-by: Katrin Leinweber <9948149+katrinleinweber@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This patch defines an internal Spark interface for reporting shuffle metrics and uses that in shuffle reader. Before this patch, shuffle metrics is tied to a specific implementation (using a thread local temporary data structure and accumulators). After this patch, callers that define their own shuffle RDDs can create a custom metrics implementation.
With this patch, we would be able to create a better metrics for the SQL layer, e.g. reporting shuffle metrics in the SQL UI, for each exchange operator.
Note that I'm separating read side and write side implementations, as they are very different, to simplify code review. Write side change is at https://github.com/apache/spark/pull/23106
## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.
Closes#23105 from rxin/SPARK-26140.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
Fix Decimal `toScalaBigInt` and `toJavaBigInteger` used to only work for decimals not fitting long.
## How was this patch tested?
Added test to DecimalSuite.
Closes#23022 from juliuszsompolski/SPARK-26038.
Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose new options for CSV datasource - `lineSep` similar to Text and JSON datasource. The option allows to specify custom line separator of maximum length of 2 characters (because of a restriction in `uniVocity` parser). New option can be used in reading and writing CSV files.
## How was this patch tested?
Added a few tests with custom `lineSep` for enabled/disabled `multiLine` in read as well as tests in write. Also I added roundtrip tests.
Closes#23080 from MaxGekk/csv-line-sep.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
the pr #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError `is thrown.
so apply for memory using `MemoryConsumer. allocatePage `when catch exception, use `SparkOutOfMemoryError `instead of `OutOfMemoryError`
## How was this patch tested?
N / A
Closes#23084 from heary-cao/SparkOutOfMemoryError.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
GROUP BY treats -0.0 and 0.0 as different values which is unlike hive's behavior.
In addition current behavior with codegen is unpredictable (see example in JIRA ticket).
## What changes were proposed in this pull request?
In Platform.putDouble/Float() checking if the value is -0.0, and if so replacing with 0.0.
This is used by UnsafeRow so it won't have -0.0 values.
## How was this patch tested?
Added tests
Closes#23043 from adoron/adoron-spark-26021-replace-minus-zero-with-zero.
Authored-by: Alon Doron <adoron@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is an addendum patch for SPARK-26129 that defines the edge case behavior for QueryPlanningTracker.topRulesByTime.
## How was this patch tested?
Added unit tests for each behavior.
Closes#23110 from rxin/SPARK-26129-1.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
When I ran spark-shell on JDK11+28(2018-09-25), It failed with the error below.
```
Exception in thread "main" java.lang.ExceptionInInitializerError
at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:80)
at org.apache.hadoop.security.SecurityUtil.getAuthenticationMethod(SecurityUtil.java:611)
at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:273)
at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:261)
at org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:791)
at org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:761)
at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:634)
at org.apache.spark.util.Utils$.$anonfun$getCurrentUserName$1(Utils.scala:2427)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.util.Utils$.getCurrentUserName(Utils.scala:2427)
at org.apache.spark.SecurityManager.<init>(SecurityManager.scala:79)
at org.apache.spark.deploy.SparkSubmit.secMgr$lzycompute$1(SparkSubmit.scala:359)
at org.apache.spark.deploy.SparkSubmit.secMgr$1(SparkSubmit.scala:359)
at org.apache.spark.deploy.SparkSubmit.$anonfun$prepareSubmitEnvironment$9(SparkSubmit.scala:367)
at scala.Option.map(Option.scala:146)
at org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:367)
at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:143)
at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:927)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:936)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.StringIndexOutOfBoundsException: begin 0, end 3, length 2
at java.base/java.lang.String.checkBoundsBeginEnd(String.java:3319)
at java.base/java.lang.String.substring(String.java:1874)
at org.apache.hadoop.util.Shell.<clinit>(Shell.java:52)
```
This is a Hadoop issue that fails to parse some java.version. It has been fixed from Hadoop-2.7.4(see [HADOOP-14586](https://issues.apache.org/jira/browse/HADOOP-14586)).
Note, Hadoop-2.7.5 or upper have another problem with Spark ([SPARK-25330](https://issues.apache.org/jira/browse/SPARK-25330)). So upgrading to 2.7.4 would be fine for now.
## How was this patch tested?
Existing tests.
Closes#23101 from tasanuma/SPARK-26134.
Authored-by: Takanobu Asanuma <tasanuma@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
The corrupt column specified via JSON/CSV option *columnNameOfCorruptRecord* must have the `string` type and be `nullable`. This has been already checked in `DataFrameReader`.`csv`/`json` and in `Json`/`CsvFileFormat` but not in `from_json`/`from_csv`. The PR adds such checks inside functions as well.
## How was this patch tested?
Added tests to `Json`/`CsvExpressionSuite` for checking type of the corrupt column. They don't check the `nullable` property because `schema` is forcibly casted to nullable.
Closes#23070 from MaxGekk/verify-corrupt-column-csv-json.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
When doing typed aggregation on a Dataset, for struct key type, the key attribute is named as "key". But for non-struct type, the key attribute is named as "value". This key attribute should also be named as "key" for non-struct type.
## How was this patch tested?
Added test.
Closes#23054 from viirya/SPARK-26085.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
An input without valid JSON tokens on the root level will be treated as a bad record, and handled according to `mode`. Previously such input was converted to `null`. After the changes, the input is converted to a row with `null`s in the `PERMISSIVE` mode according the schema. This allows to remove a code in the `from_json` function which can produce `null` as result rows.
## How was this patch tested?
It was tested by existing test suites. Some of them I have to modify (`JsonSuite` for example) because previously bad input was just silently ignored. For now such input is handled according to specified `mode`.
Closes#22938 from MaxGekk/json-nulls.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose:
- new SQL config `spark.sql.debug.maxToStringFields` to control maximum number fields up to which `truncatedString` cuts its input sequences.
- Moving `truncatedString` out of `core` to `sql/catalyst` because it is used only in the `sql/catalyst` packages for restricting number of fields converted to strings from `TreeNode` and expressions of`StructType`.
## How was this patch tested?
Added a test to `QueryExecutionSuite` to check that `spark.sql.debug.maxToStringFields` impacts to behavior of `truncatedString`.
Closes#23039 from MaxGekk/truncated-string-catalyst.
Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
We currently don't have good visibility into query planning time (analysis vs optimization vs physical planning). This patch adds a simple utility to track the runtime of various rules and various planning phases.
## How was this patch tested?
Added unit tests and end-to-end integration tests.
Closes#23096 from rxin/SPARK-26129.
Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
## What changes were proposed in this pull request?
This change fixes a particular scenario where default spark SQL can't encode (thrift) types that are generated by twitter scrooge. These types are a trait that extends `scala.ProductX` with a constructor defined only in a companion object, rather than a actual case class. The actual case class used is child class, but that type is almost never referred to in code. The type has no corresponding constructor symbol and causes an exception. For all other purposes, these classes act just like case classes, so it is unfortunate that spark SQL can't serialize them nicely as it can actual case classes. For an full example of a scrooge codegen class, see https://gist.github.com/anonymous/ba13d4b612396ca72725eaa989900314.
This change catches the case where the type has no constructor but does have an `apply` method on the type's companion object. This allows for thrift types to be serialized/deserialized with implicit encoders the same way as normal case classes. This fix had to be done in three places where the constructor is assumed to be an actual constructor:
1) In serializing, determining the schema for the dataframe relies on inspecting its constructor (`ScalaReflection.constructParams`). Here we fall back to using the companion constructor arguments.
2) In deserializing or evaluating, in the java codegen ( `NewInstance.doGenCode`), the type couldn't be constructed with the new keyword. If there is no constructor, we change the constructor call to try the companion constructor.
3) In deserializing or evaluating, without codegen, the constructor is directly invoked (`NewInstance.constructor`). This was fixed with scala reflection to get the actual companion apply method.
The return type of `findConstructor` was changed because the companion apply method constructor can't be represented as a `java.lang.reflect.Constructor`.
There might be situations in which this approach would also fail in a new way, but it does at a minimum work for the specific scrooge example and will not impact cases that were already succeeding prior to this change
Note: this fix does not enable using scrooge thrift enums, additional work for this is necessary. With this patch, it seems like you could patch `com.twitter.scrooge.ThriftEnum` to extend `_root_.scala.Product1[Int]` with `def _1 = value` to get spark's implicit encoders to handle enums, but I've yet to use this method myself.
Note: I previously opened a PR for this issue, but only was able to fix case 1) there: https://github.com/apache/spark/pull/18766
## How was this patch tested?
I've fixed all 3 cases and added two tests that use a case class that is similar to scrooge generated one. The test in ScalaReflectionSuite checks 1), and the additional asserting in ObjectExpressionsSuite checks 2) and 3).
Closes#23062 from drewrobb/SPARK-8288.
Authored-by: Drew Robb <drewrobb@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to pass the CSV option `encoding`/`charset` to `uniVocity` parser to allow parsing CSV files in different encodings when `multiLine` is enabled. The value of the option is passed to the `beginParsing` method of `CSVParser`.
## How was this patch tested?
Added new test to `CSVSuite` for different encodings and enabled/disabled header.
Closes#23091 from MaxGekk/csv-miltiline-encoding.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This PR fixes an exception in `AggregateExpression.references` called on unresolved expressions. It implements the solution proposed in [SPARK-26084](https://issues.apache.org/jira/browse/SPARK-26084), a minor refactoring that removes the unnecessary dependence on `AttributeSet.toSeq`, which requires expression IDs and, therefore, can only execute successfully for resolved expressions.
The refactored implementation is both simpler and faster, eliminating the conversion of a `Set` to a
`Seq` and back to `Set`.
## How was this patch tested?
Added a new test based on the failing case in [SPARK-26084](https://issues.apache.org/jira/browse/SPARK-26084).
hvanhovell
Closes#23075 from ssimeonov/ss_SPARK-26084.
Authored-by: Simeon Simeonov <sim@fastignite.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
## What changes were proposed in this pull request?
Extend the `ReplaceNullWithFalse` optimizer rule introduced in SPARK-25860 (https://github.com/apache/spark/pull/22857) to also support optimizing predicates in higher-order functions of `ArrayExists`, `ArrayFilter`, `MapFilter`.
Also rename the rule to `ReplaceNullWithFalseInPredicate` to better reflect its intent.
Example:
```sql
select filter(a, e -> if(e is null, null, true)) as b from (
select array(null, 1, null, 3) as a)
```
The optimized logical plan:
**Before**:
```
== Optimized Logical Plan ==
Project [filter([null,1,null,3], lambdafunction(if (isnull(lambda e#13)) null else true, lambda e#13, false)) AS b#9]
+- OneRowRelation
```
**After**:
```
== Optimized Logical Plan ==
Project [filter([null,1,null,3], lambdafunction(if (isnull(lambda e#13)) false else true, lambda e#13, false)) AS b#9]
+- OneRowRelation
```
## How was this patch tested?
Added new unit test cases to the `ReplaceNullWithFalseInPredicateSuite` (renamed from `ReplaceNullWithFalseSuite`).
Closes#23079 from rednaxelafx/catalyst-master.
Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The build has a lot of deprecation warnings. Some are new in Scala 2.12 and Java 11. We've fixed some, but I wanted to take a pass at fixing lots of easy miscellaneous ones here.
They're too numerous and small to list here; see the pull request. Some highlights:
- `BeanInfo` is deprecated in 2.12, and BeanInfo classes are pretty ancient in Java. Instead, case classes can explicitly declare getters
- Eta expansion of zero-arg methods; foo() becomes () => foo() in many cases
- Floating-point Range is inexact and deprecated, like 0.0 to 100.0 by 1.0
- finalize() is finally deprecated (just needs to be suppressed)
- StageInfo.attempId was deprecated and easiest to remove here
I'm not now going to touch some chunks of deprecation warnings:
- Parquet deprecations
- Hive deprecations (particularly serde2 classes)
- Deprecations in generated code (mostly Thriftserver CLI)
- ProcessingTime deprecations (we may need to revive this class as internal)
- many MLlib deprecations because they concern methods that may be removed anyway
- a few Kinesis deprecations I couldn't figure out
- Mesos get/setRole, which I don't know well
- Kafka/ZK deprecations (e.g. poll())
- Kinesis
- a few other ones that will probably resolve by deleting a deprecated method
## How was this patch tested?
Existing tests, including manual testing with the 2.11 build and Java 11.
Closes#23065 from srowen/SPARK-26090.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Due to implementation limitation, currently Spark can't compare or do equality check between map types. As a result, map values can't appear in EQUAL or comparison expressions, can't be grouping key, etc.
The more important thing is, map loop up needs to do equality check of the map key, and thus can't support map as map key when looking up values from a map. Thus it's not useful to have map as map key.
This PR proposes to stop users from creating maps using map type as key. The list of expressions that are updated: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `MapConcat`, `TransformKeys`. I manually checked all the places that create `MapType`, and came up with this list.
Note that, maps with map type key still exist, via reading from parquet files, converting from scala/java map, etc. This PR is not to completely forbid map as map key, but to avoid creating it by Spark itself.
Motivation: when I was trying to fix the duplicate key problem, I found it's impossible to do it with map type map key. I think it's reasonable to avoid map type map key for builtin functions.
## How was this patch tested?
updated test
Closes#23045 from cloud-fan/map-key.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Following [SPARK-26024](https://issues.apache.org/jira/browse/SPARK-26024), I noticed the number of elements in each partition after repartitioning using `df.repartitionByRange` can vary for the same setup:
```scala
// Shuffle numbers from 0 to 1000, and make a DataFrame
val df = Random.shuffle(0.to(1000)).toDF("val")
// Repartition it using 3 partitions
// Sum up number of elements in each partition, and collect it.
// And do it several times
for (i <- 0 to 9) {
var counts = df.repartitionByRange(3, col("val"))
.mapPartitions{part => Iterator(part.size)}
.collect()
println(counts.toList)
}
// -> the number of elements in each partition varies
```
This is expected as for performance reasons this method uses sampling to estimate the ranges (with default size of 100). Hence, the output may not be consistent, since sampling can return different values. But documentation was not mentioning it at all, leading to misunderstanding.
## What changes were proposed in this pull request?
Update the documentation (Spark & PySpark) to mention the impact of `spark.sql.execution.rangeExchange.sampleSizePerPartition` on the resulting partitioned DataFrame.
Closes#23025 from JulienPeloton/SPARK-26024.
Authored-by: Julien <peloton@lal.in2p3.fr>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
The following 5 functions were removed from branch-2.4:
- map_entries
- map_filter
- transform_values
- transform_keys
- map_zip_with
We should update the since version to 3.0.0.
## How was this patch tested?
Existing tests.
Closes#23082 from ueshin/issues/SPARK-26112/since.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This restores scaladoc artifact generation, which got dropped with the Scala 2.12 update. The change looks large, but is almost all due to needing to make the InterfaceStability annotations top-level classes (i.e. `InterfaceStability.Stable` -> `Stable`), unfortunately. A few inner class references had to be qualified too.
Lots of scaladoc warnings now reappear. We can choose to disable generation by default and enable for releases, later.
## How was this patch tested?
N/A; build runs scaladoc now.
Closes#23069 from srowen/SPARK-26026.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
[Hive 2.3.4 is released on Nov. 7th](https://hive.apache.org/downloads.html#7-november-2018-release-234-available). This PR aims to support that version.
## How was this patch tested?
Pass the Jenkins with the updated version
Closes#23059 from dongjoon-hyun/SPARK-26091.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Since [Spark 1.6.0](56d7da14ab (diff-6f38a103058a6e233b7ad80718452387R96)), there was a redundant '}' character in CTAS string plan's database argument string; `default}`. This PR aims to fix it.
**BEFORE**
```scala
scala> sc.version
res1: String = 1.6.0
scala> sql("create table t as select 1").explain
== Physical Plan ==
ExecutedCommand CreateTableAsSelect [Database:default}, TableName: t, InsertIntoHiveTable]
+- Project [1 AS _c0#3]
+- OneRowRelation$
```
**AFTER**
```scala
scala> sql("create table t as select 1").explain
== Physical Plan ==
Execute CreateHiveTableAsSelectCommand CreateHiveTableAsSelectCommand [Database:default, TableName: t, InsertIntoHiveTable]
+- *(1) Project [1 AS 1#4]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Manual.
Closes#23064 from dongjoon-hyun/SPARK-FIX.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
Events are dispatched on a separate thread, so need to wait for them to be
actually delivered before checking that the listener got them.
Closes#23050 from vanzin/SPARK-26079.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Use CheckpointFileManager to write the streaming `metadata` file so that the `metadata` file will never be a partial file.
## How was this patch tested?
Jenkins
Closes#23060 from zsxwing/SPARK-26092.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
remove invalid comment as we don't use it anymore
More details: https://github.com/apache/spark/pull/22976#discussion_r233764857
## How was this patch tested?
N/A
Closes#23044 from heary-cao/followUpOrdering.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This is a follow-up of #20393.
We should read the conf `"spark.sql.sort.enableRadixSort"` from `SQLConf` instead of `SparkConf`, i.e., use `SQLConf.get.enableRadixSort` instead of `SparkEnv.get.conf.get(SQLConf.RADIX_SORT_ENABLED)`, otherwise the config is never read.
## How was this patch tested?
Existing tests.
Closes#23046 from ueshin/issues/SPARK-23207/conf.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In SPARK-24865 `AnalysisBarrier` was removed and in order to improve resolution speed, the `analyzed` flag was (re-)introduced in order to process only plans which are not yet analyzed. This should not be the case when performing attribute deduplication as in that case we need to transform also the plans which were already analyzed, otherwise we can miss to rewrite some attributes leading to invalid plans.
## How was this patch tested?
added UT
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23035 from mgaido91/SPARK-26057.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.
We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.
## How was this patch tested?
existing tests
Closes#22967 from dbtsai/scala2.12.
Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
## What changes were proposed in this pull request?
Add scala and java lint check rules to ban the usage of `throw new xxxErrors` and fix up all exists instance followed by https://github.com/apache/spark/pull/22989#issuecomment-437939830. See more details in https://github.com/apache/spark/pull/22969.
## How was this patch tested?
Local test with lint-scala and lint-java.
Closes#22989 from xuanyuanking/SPARK-25986.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
…. Other related changes to get JDK 11 working, to test
## What changes were proposed in this pull request?
- Access `sun.misc.Cleaner` (Java 8) and `jdk.internal.ref.Cleaner` (JDK 9+) by reflection (note: the latter only works if illegal reflective access is allowed)
- Access `sun.misc.Unsafe.invokeCleaner` in Java 9+ instead of `sun.misc.Cleaner` (Java 8)
In order to test anything on JDK 11, I also fixed a few small things, which I include here:
- Fix minor JDK 11 compile issues
- Update scala plugin, Jetty for JDK 11, to facilitate tests too
This doesn't mean JDK 11 tests all pass now, but lots do. Note also that the JDK 9+ solution for the Cleaner has a big caveat.
## How was this patch tested?
Existing tests. Manually tested JDK 11 build and tests, and tests covering this change appear to pass. All Java 8 tests should still pass, but this change alone does not achieve full JDK 11 compatibility.
Closes#22993 from srowen/SPARK-24421.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>