Commit graph

29209 commits

Author SHA1 Message Date
Cheng Su 3a361cd837 [SPARK-34253][SQL] Object hash aggregate should not fallback if no more input rows
### What changes were proposed in this pull request?

Object hash aggregate will fallback to sort-based aggregation based on number of keys seen so far [0]. The default config threshold is 128 (spark.sql.objectHashAggregate.sortBased.fallbackThreshold in [1]). There's an edge case we can do better, where we do not fallback if there's no more input rows. Suppose the task only has 128 group-by keys in hash ma, we don't need to fallback in this case, and we can save the extra sort. This is an rare edge case in production, but it can happen.

[0]: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala#L161

[1]: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L1615

### Why are the changes needed?

To avoid unnecessary sort in query. Save resource.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Add a unit test to verify task fallback or not is challenging. Given the change is pretty minor, besides relying on existing test in `ObjectHashAggregateSuite.scala`, I manually ran the followed query, and verified in debug mode that the code path for fallback was not executed. And verified the code path for fallback was executed without this change.

```
withSQLConf(
  SQLConf.USE_OBJECT_HASH_AGG.key -> "true",
  SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "1") {
  Seq.fill(1)(Tuple1(Array.empty[Int]))
    .toDF("c0")
    .groupBy(lit(1))
    .agg(typed_count($"c0"), max($"c0")).collect()
}
```

Closes #31353 from c21/object-hash-fallback.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 15:18:54 +00:00
MrPowers 9ed0e3cebf [SPARK-34165][SQL] Add count_distinct as an option to Dataset#summary
### What changes were proposed in this pull request?

Add `count_distinct` as an option argument to Dataset#summary (the method already supports count, min, max, etc.)

### Why are the changes needed?

The `summary()` method is used for lightweight exploratory data analysis.  A distinct count of all the columns is one of the most common exploratory data analysis queries.

Distinct counts can be expensive, so this shouldn't be enabled by default.  The proposed implementation is completely backwards compatible.

### Does this PR introduce _any_ user-facing change?

Yes, users can now call `df.summary("count_distinct")`, which wasn't an option before.  Users can still call `df.summary()` without any arguments and the output is the same.  `count_distinct` was not added as one of the `defaultStatistics`.

### How was this patch tested?

Unit tests.

### Additional comments

If this idea is accepted, we should add a PySpark implementation in this PR, as suggested by zero323.

Closes #31254 from MrPowers/SPARK-34165.

Authored-by: MrPowers <matthewkevinpowers@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-28 08:38:01 -06:00
yangjie01 15445a8d9e [SPARK-34275][CORE][SQL][MLLIB] Replaces filter and size with count
### What changes were proposed in this pull request?
Use `count` to simplify `find + size(or length)` operation, it's semantically consistent, but looks simpler.

**Before**
```
seq.filter(p).size
```

**After**
```
seq.count(p)
```

### Why are the changes needed?
Code Simpilefications.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31374 from LuciferYang/SPARK-34275.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 15:27:07 +09:00
Max Gekk d242166b8f [SPARK-34262][SQL] Refresh cached data of v1 table in ALTER TABLE .. SET LOCATION
### What changes were proposed in this pull request?
Invoke `CatalogImpl.refreshTable()` in v1 implementation of the `ALTER TABLE .. SET LOCATION` command to refresh cached table data.

### Why are the changes needed?
The example below portraits the issue:

- Create a source table:
```sql
spark-sql> CREATE TABLE src_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> INSERT INTO src_tbl PARTITION (part=0) SELECT 0;
spark-sql> SHOW TABLE EXTENDED LIKE 'src_tbl' PARTITION (part=0);
default	src_tbl	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0
...
```
- Set new location for the empty partition (part=0):
```sql
spark-sql> CREATE TABLE dst_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> ALTER TABLE dst_tbl ADD PARTITION (part=0);
spark-sql> INSERT INTO dst_tbl PARTITION (part=1) SELECT 1;
spark-sql> CACHE TABLE dst_tbl;
spark-sql> SELECT * FROM dst_tbl;
1	1
spark-sql> ALTER TABLE dst_tbl PARTITION (part=0) SET LOCATION '/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0';
spark-sql> SELECT * FROM dst_tbl;
1	1
```
The last query does not return new loaded data.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works correctly:
```sql
spark-sql> ALTER TABLE dst_tbl PARTITION (part=0) SET LOCATION '/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0';
spark-sql> SELECT * FROM dst_tbl;
0	0
1	1
```

### How was this patch tested?
Added new test to `org.apache.spark.sql.hive.CachedTableSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite"
```

Closes #31361 from MaxGekk/refresh-cache-set-location.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 15:05:22 +09:00
beliefer b12e9a4ea6 [SPARK-33542][SQL][FOLLOWUP] Group exception messages in catalyst/catalog
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/30870.
Maybe some contributors don't know the job and added some exception by the old way.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31312 from beliefer/SPARK-33542-followup.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 05:15:57 +00:00
Angerszhuuuu 850990f40e [SPARK-34238][SQL] Unify output of SHOW PARTITIONS and pass output attributes properly
### What changes were proposed in this pull request?
Passing around the output attributes should have more benefits like keeping the expr ID unchanged to avoid bugs when we apply more operators above the command output dataframe.

This PR keep SHOW PARTITIONS command's output attribute exprId unchanged.
And benefit for https://issues.apache.org/jira/browse/SPARK-34238
### Why are the changes needed?
 Keep SHOW PARTITIONS command's output attribute exprid unchanged.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added UT

Closes #31341 from AngersZhuuuu/SPARK-34238.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 05:13:19 +00:00
Yuming Wang 01d11da84e [SPARK-34268][SQL][DOCS] Correct the documentation of the concat_ws function
### What changes were proposed in this pull request?

This pr correct the documentation of the `concat_ws` function.

### Why are the changes needed?

`concat_ws` doesn't need any str or array(str) arguments:
```
scala> sql("""select concat_ws("s")""").show
+------------+
|concat_ws(s)|
+------------+
|            |
+------------+
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

```
 build/sbt  "sql/testOnly *.ExpressionInfoSuite"
```

Closes #31370 from wangyum/SPARK-34268.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 14:06:36 +09:00
Chao Sun 6ec3cf6219 [SPARK-34271][SQL] Use majorMinorPatchVersion for Hive version parsing
### What changes were proposed in this pull request?

Use `majorMinorPatchVersion` to check major & minor version in `IsolatedClientLoader.hiveVersion`.

### Why are the changes needed?

Currently `IsolatedClientLoader.hiveVersion` needs to enumerate all Hive patch versions. Therefore, whenever we upgrade Hive version we'd need to remember to update the method as well. It would be better if we just check major & minor version.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

This is a refactoring and relies on existing tests.

Closes #31371 from sunchao/replace-hive-version.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 14:00:10 +09:00
Linhong Liu cf1400c8dd [SPARK-34260][SQL] Fix UnresolvedException when creating temp view twice
### What changes were proposed in this pull request?
In PR #30140, it will compare new and old plans when replacing view and uncache data
if the view has changed. But the compared new plan is not analyzed which will cause
`UnresolvedException` when calling `sameResult`. So in this PR, we use the analyzed
plan to compare to fix this problem.

### Why are the changes needed?
bug fix

### Does this PR introduce _any_ user-facing change?
no

### How was this patch tested?
newly added tests

Closes #31360 from linhongliu-db/SPARK-34260.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 20:59:23 -08:00
Chircu 829f118f98 [SPARK-33867][SQL] Instant and LocalDate values aren't handled when generating SQL queries
### What changes were proposed in this pull request?

When generating SQL queries only the old date time API types are handled for values in org.apache.spark.sql.jdbc.JdbcDialect#compileValue. If the new API is used (spark.sql.datetime.java8API.enabled=true) Instant and LocalDate values are not quoted and errors are thrown. The change proposed is to handle Instant and LocalDate values the same way that Timestamp and Date are.

### Why are the changes needed?

In the current state if an Instant is used in a filter, an exception will be thrown.
Ex (dataset was read from PostgreSQL): dataset.filter(current_timestamp().gt(col(VALID_FROM)))
Stacktrace (the T11 is from an instant formatted like yyyy-MM-dd'T'HH:mm:ss.SSSSSS'Z'):
Caused by: org.postgresql.util.PSQLException: ERROR: syntax error at or near "T11"Caused by: org.postgresql.util.PSQLException: ERROR: syntax error at or near "T11"  Position: 285 at org.postgresql.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2103) at org.postgresql.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:1836) at org.postgresql.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:257) at org.postgresql.jdbc2.AbstractJdbc2Statement.execute(AbstractJdbc2Statement.java:512) at org.postgresql.jdbc2.AbstractJdbc2Statement.executeWithFlags(AbstractJdbc2Statement.java:388) at org.postgresql.jdbc2.AbstractJdbc2Statement.executeQuery(AbstractJdbc2Statement.java:273) at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(JDBCRDD.scala:304) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) at org.apache.spark.scheduler.Task.run(Task.scala:127) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) at java.base/java.lang.Thread.run(Thread.java:834)

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Test added

Closes #31148 from cristichircu/SPARK-33867.

Lead-authored-by: Chircu <chircu@arezzosky.com>
Co-authored-by: Cristi Chircu <chircu@arezzosky.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-01-28 11:58:20 +09:00
“attilapiros” 0dedf24cd0 [SPARK-34154][YARN] Extend LocalityPlacementStrategySuite's test with a timeout
### What changes were proposed in this pull request?

This PR extends the `handle large number of containers and tasks (SPARK-18750)` test with a time limit and in case of timeout it saves the stack trace of the running thread to provide extra information about the reason why it got stuck.

### Why are the changes needed?

This is a flaky test which sometime runs for hours without stopping.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

I checked it with a temporary code change: by adding a `Thread.sleep` to `LocalityPreferredContainerPlacementStrategy#expectedHostToContainerCount`.

The stack trace showed the correct method:

```
[info] LocalityPlacementStrategySuite:
[info] - handle large number of containers and tasks (SPARK-18750) *** FAILED *** (30 seconds, 26 milliseconds)
[info]   Failed with an exception or a timeout at thread join:
[info]
[info]   java.lang.RuntimeException: Timeout at waiting for thread to stop (its stack trace is added to the exception)
[info]   	at java.lang.Thread.sleep(Native Method)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.$anonfun$expectedHostToContainerCount$1(LocalityPreferredContainerPlacementStrategy.scala:198)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy$$Lambda$281/381161906.apply(Unknown Source)
[info]   	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[info]   	at scala.collection.TraversableLike$$Lambda$16/322836221.apply(Unknown Source)
[info]   	at scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:234)
[info]   	at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:468)
[info]   	at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:468)
[info]   	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
[info]   	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
[info]   	at scala.collection.AbstractTraversable.map(Traversable.scala:108)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.expectedHostToContainerCount(LocalityPreferredContainerPlacementStrategy.scala:188)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.localityOfRequestedContainers(LocalityPreferredContainerPlacementStrategy.scala:112)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite.org$apache$spark$deploy$yarn$LocalityPlacementStrategySuite$$runTest(LocalityPlacementStrategySuite.scala:94)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite$$anon$1.run(LocalityPlacementStrategySuite.scala:40)
[info]   	at java.lang.Thread.run(Thread.java:748) (LocalityPlacementStrategySuite.scala:61)
...
```

Closes #31363 from attilapiros/SPARK-34154.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 08:04:25 +09:00
Holden Karau 9d83d62f14 [SPARK-34193][CORE] TorrentBroadcast block manager decommissioning race fix
### What changes were proposed in this pull request?

Allow broadcast blocks to be put during decommissioning since migrations don't apply to them and they may be stored as part of job exec.

### Why are the changes needed?

Potential race condition.

### Does this PR introduce _any_ user-facing change?

Removal of race condition.

### How was this patch tested?

New unit test.

Closes #31298 from holdenk/SPARK-34193-torrentbroadcast-blockmanager-decommissioning-potential-race-condition.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 06:15:35 +09:00
neko f1bc37e624 [SPARK-34221][WEBUI] Ensure if a stage fails in the UI page, the corresponding error message can be displayed correctly
### What changes were proposed in this pull request?
Ensure that if a stage fails in the UI page, the corresponding error message can be displayed correctly.

### Why are the changes needed?
errormessage is not handled properly in JavaScript. If the 'at' is not exist, the error message on the page will be blank.
I made wochanges,
1. `msg.indexOf("at")` => `msg.indexOf("\n")`

![image](https://user-images.githubusercontent.com/52202080/105663531-7362cb00-5f0d-11eb-87fd-008ed65c33ca.png)

  As shows ablove, truncated at the 'at' position will result in a strange abstract of the error message. If there is a `\n` worit is more reasonable to truncate at the '\n' position.

2. If the `\n` does not exist check whether the msg  is more than 100. If true, then truncate the display to avoid too long error message

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual test shows as belows, just a js change:

before modified:
![problem](https://user-images.githubusercontent.com/52202080/105712153-661cff00-5f54-11eb-80bf-e33c323c4e55.png)

after modified
![after mdified](https://user-images.githubusercontent.com/52202080/105712180-6c12e000-5f54-11eb-8998-ff8bc8a0a503.png)

Closes #31314 from akiyamaneko/error_message_display_empty.

Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 10:01:57 -08:00
Max Gekk 1318be7ee9 [SPARK-34267][SQL] Remove refreshTable() from SessionState
### What changes were proposed in this pull request?
Remove `SessionState.refreshTable()` and modify the tests where the method is used.

### Why are the changes needed?
There are already 2 methods with the same name in:
- `SessionCatalog`
- `CatalogImpl`

One more method in `SessionState` does not give any benefits. By removing it, we can improve code maintenance.

### Does this PR introduce _any_ user-facing change?
Should not because `SessionState` is an internal class.

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *MetastoreDataSourcesSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveOrcQuerySuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveParquetMetastoreSuite"
```

Closes #31366 from MaxGekk/remove-refreshTable-from-SessionState.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 09:43:59 -08:00
Wenchen Fan 2dbb7d5af8 [SPARK-34212][SQL][FOLLOWUP] Refine the behavior of reading parquet non-decimal fields as decimal
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/31319 .

When reading parquet int/long as decimal, the behavior should be the same as reading int/long and then cast to the decimal type. This PR changes to the expected behavior.

When reading parquet binary as decimal, we don't really know how to interpret the binary (it may from a string), and should fail. This PR changes to the expected behavior.

### Why are the changes needed?

To make the behavior more sane.

### Does this PR introduce _any_ user-facing change?

Yes, but it's a followup.

### How was this patch tested?

updated test

Closes #31357 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 09:34:31 -08:00
Kent Yao 5718d64f31 [SPARK-34083][SQL] Using TPCDS original definitions for char/varchar columns
### What changes were proposed in this pull request?

This PR changes the column types in the table definitions of `TPCDSBase` from string to char and varchar, with respect to the original definitions for char/varchar columns in the official doc - [TPC-DS_v2.9.0](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Comply with both TPCDS standard and ANSI, and using string will get wrong results with those TPCDS queries

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

plan stability check

Closes #31012 from yaooqinn/tpcds.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 17:51:49 +08:00
HyukjinKwon 1217c8b418 Revert "[SPARK-31168][SPARK-33913][BUILD] Upgrade Scala to 2.12.13 and Kafka to 2.7.0"
This reverts commit a65e86a65e.
2021-01-27 17:03:15 +09:00
Erik Krogen b2c104bd87 [SPARK-34231][AVRO][TEST] Make proper use of resource file within AvroSuite test case
### What changes were proposed in this pull request?
Change `AvroSuite."Ignore corrupt Avro file if flag IGNORE_CORRUPT_FILES"` to use `episodesAvro`, which is loaded as a resource using the classloader, instead of trying to read `episodes.avro` directly from a relative file path.

### Why are the changes needed?
This is the proper way to read resource files, and currently this test will fail when called from my IntelliJ IDE, though it will succeed when called from Maven/sbt, presumably due to different working directory handling.

### Does this PR introduce _any_ user-facing change?
No, unit test only.

### How was this patch tested?
Previous failure from IntelliJ:
```
Source 'src/test/resources/episodes.avro' does not exist
java.io.FileNotFoundException: Source 'src/test/resources/episodes.avro' does not exist
	at org.apache.commons.io.FileUtils.checkFileRequirements(FileUtils.java:1405)
	at org.apache.commons.io.FileUtils.copyFile(FileUtils.java:1072)
	at org.apache.commons.io.FileUtils.copyFile(FileUtils.java:1040)
	at org.apache.spark.sql.avro.AvroSuite.$anonfun$new$34(AvroSuite.scala:397)
	at org.apache.spark.sql.avro.AvroSuite.$anonfun$new$34$adapted(AvroSuite.scala:388)
```
Now it succeeds.

Closes #31332 from xkrogen/xkrogen-SPARK-34231-avrosuite-testfix.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-27 16:16:33 +09:00
Max Gekk 0d08e22bc7 [SPARK-34251][SQL] Fix table stats calculation by TRUNCATE TABLE
### What changes were proposed in this pull request?
1. Take into account the SQL config `spark.sql.statistics.size.autoUpdate.enabled` in the `TRUNCATE TABLE` command as other commands do.
2. Re-calculate actual table size in fs. Before the changes, `TRUNCATE TABLE` always sets table size to 0 in stats.

### Why are the changes needed?
This fixes the bug that is demonstrated by the example:
1. Create a partitioned table with 2 non-empty partitions:
```sql
spark-sql> CREATE TABLE tbl (c0 int, part int) PARTITIONED BY (part);
spark-sql> INSERT INTO tbl PARTITION (part=0) SELECT 0;
spark-sql> INSERT INTO tbl PARTITION (part=1) SELECT 1;
spark-sql> ANALYZE TABLE tbl COMPUTE STATISTICS;
spark-sql> DESCRIBE TABLE EXTENDED tbl;
...
Statistics	4 bytes, 2 rows
...
```
2. Truncate only one partition:
```sql
spark-sql> TRUNCATE TABLE tbl PARTITION (part=1);
spark-sql> SELECT * FROM tbl;
0	0
```
3. The table is still non-empty but `TRUNCATE TABLE` reseted stats:
```
spark-sql> DESCRIBE TABLE EXTENDED tbl;
...
Statistics	0 bytes, 0 rows
...
```

### Does this PR introduce _any_ user-facing change?
It could impact on performance of following queries.

### How was this patch tested?
Added new test to `StatisticsCollectionSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *StatisticsCollectionSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *StatisticsSuite"
```

Closes #31350 from MaxGekk/fix-stats-in-trunc-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 07:02:04 +00:00
Kent Yao 764582c07a [SPARK-34233][SQL] FIX NPE for char padding in binary comparison
### What changes were proposed in this pull request?

we need to check whether the `lit` is null  before calling `numChars`

### Why are the changes needed?

fix an obvious NPE bug

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

new tests

Closes #31336 from yaooqinn/SPARK-34233.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 14:59:53 +08:00
Kent Yao 91ca21d700 [SPARK-34236][SQL] Fix v2 Overwrite w/ null static partition raise Cannot translate expression to source filter: null
### What changes were proposed in this pull request?

For v2 static partitions overwriting, we use `EqualTo ` to generate the `deleteExpr`

This is not right for null partition values, and cause the problem like below because `ConstantFolding` converts it to lit(null)

```scala
SPARK-34223: static partition with null raise NPE *** FAILED *** (19 milliseconds)
[info]   org.apache.spark.sql.AnalysisException: Cannot translate expression to source filter: null
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.$anonfun$applyOrElse$1(V2Writes.scala:50)
[info]   at scala.collection.immutable.List.flatMap(List.scala:366)
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.applyOrElse(V2Writes.scala:47)
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.applyOrElse(V2Writes.scala:39)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:317)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:73)
```

The right way is to use EqualNullSafe instead to delete the null partitions.

### Why are the changes needed?

bugfix

### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

an original test to new place

Closes #31339 from yaooqinn/SPARK-34236.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 12:05:50 +08:00
Ruifeng Zheng 2c4e4f8412 [SPARK-34189][ML] w2v findSynonyms optimization
### What changes were proposed in this pull request?
1, use Guavaording instead of BoundedPriorityQueue;
2, use local variables;
3, avoid conversion: ml.vector -> mllib.vector

### Why are the changes needed?
this pr is about 30% faster than existing impl

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
existing testsuites

Closes #31276 from zhengruifeng/w2v_findSynonyms_opt.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2021-01-27 10:08:53 +08:00
Ruifeng Zheng 3c686708d5 [SPARK-34220][ML] BucketedRandomProjectionLSH transform optimization
### What changes were proposed in this pull request?
use GEMV instead of DOT

### Why are the changes needed?
1, better performance, could be 20% faster than existing impl
2, simplify model saving

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
existing testsuites

Closes #31313 from zhengruifeng/random_project_opt.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2021-01-27 10:05:53 +08:00
Ruifeng Zheng 56e9426a9e [SPARK-33518][ML][FOLLOWUP] MatrixFactorizationModel use GEMV
### What changes were proposed in this pull request?
1, update related doc;
2, MatrixFactorizationModel use GEMV;

### Why are the changes needed?
see performance gain in https://github.com/apache/spark/pull/30468

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
existing testsuites

Closes #31279 from zhengruifeng/als_follow_up.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2021-01-27 10:02:37 +08:00
Chao Sun abf7e81712 [SPARK-33212][FOLLOW-UP][BUILD] Bring back duplicate dependency check and add more strict Hadoop version check
### What changes were proposed in this pull request?

1. Add back Maven enforcer for duplicate dependencies check
2. More strict check on Hadoop versions which support shaded client in `IsolatedClientLoader`. To do proper version check, this adds a util function `majorMinorPatchVersion` to extract major/minor/patch version from a string.
3. Cleanup unnecessary code

### Why are the changes needed?

The Maven enforcer was removed as part of #30556. This proposes to add it back.

Also, Hadoop shaded client doesn't work in certain cases (see [these comments](https://github.com/apache/spark/pull/30701#discussion_r558522227) for details). This strictly checks that the current Hadoop version (i.e., 3.2.2 at the moment) has good support of shaded client or otherwise fallback to old unshaded ones.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #31203 from sunchao/SPARK-33212-followup.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:34:55 -08:00
Dongjoon Hyun dbf051c50a [SPARK-34212][SQL] Fix incorrect decimal reading from Parquet files
### What changes were proposed in this pull request?

This PR aims to the correctness issues during reading decimal values from Parquet files.
- For **MR** code path, `ParquetRowConverter` can read Parquet's decimal values with the original precision and scale written in the corresponding footer.
- For **Vectorized** code path, `VectorizedColumnReader` throws `SchemaColumnConvertNotSupportedException`.

### Why are the changes needed?

Currently, Spark returns incorrect results when the Parquet file's decimal precision and scale are different from the Spark's schema. This happens when there is multiple files with different decimal schema or HiveMetastore has a new schema.

**BEFORE (Simplified example for correctness)**

```scala
scala> sql("SELECT 1.0 a").write.parquet("/tmp/decimal")
scala> spark.read.schema("a DECIMAL(3,2)").parquet("/tmp/decimal").show
+----+
|   a|
+----+
|0.10|
+----+
```

This works correctly in the other data sources, `ORC/JSON/CSV`, like the following.
```scala
scala> sql("SELECT 1.0 a").write.orc("/tmp/decimal_orc")
scala> spark.read.schema("a DECIMAL(3,2)").orc("/tmp/decimal_orc").show
+----+
|   a|
+----+
|1.00|
+----+
```

**AFTER**
1. **Vectorized** path: Instead of incorrect result, we will raise an explicit exception.
```scala
scala> spark.read.schema("a DECIMAL(3,2)").parquet("/tmp/decimal").show
java.lang.UnsupportedOperationException: Schema evolution not supported.
```

2. **MR** path (complex schema or explicit configuration): Spark returns correct results.
```scala
scala> spark.read.schema("a DECIMAL(3,2), b DECIMAL(18, 3), c MAP<INT,INT>").parquet("/tmp/decimal").show
+----+-------+--------+
|   a|      b|       c|
+----+-------+--------+
|1.00|100.000|{1 -> 2}|
+----+-------+--------+

scala> spark.read.schema("a DECIMAL(3,2), b DECIMAL(18, 3), c MAP<INT,INT>").parquet("/tmp/decimal").printSchema
root
 |-- a: decimal(3,2) (nullable = true)
 |-- b: decimal(18,3) (nullable = true)
 |-- c: map (nullable = true)
 |    |-- key: integer
 |    |-- value: integer (valueContainsNull = true)
```

### Does this PR introduce _any_ user-facing change?

Yes. This fixes the correctness issue.

### How was this patch tested?

Pass with the newly added test case.

Closes #31319 from dongjoon-hyun/SPARK-34212.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:13:39 -08:00
Chao Sun c2320a43c7 [SPARK-34052][FOLLOWUP][DOC] Add document in SQL migration guide
### What changes were proposed in this pull request?

Add document for the behavior change in SPARK-34052, in SQL migration guide.

### Why are the changes needed?

Document behavior change for Spark users.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

N/A

Closes #31351 from sunchao/SPARK-34052-followup.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:11:45 -08:00
beliefer 99b6af2dd2 [SPARK-34244][SQL] Remove the Scala function version of regexp_extract_all
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/27507 implements `regexp_extract_all` and added the scala function version of it.
According https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L41-L59, it seems good for remove the scala function version. Although I think is regexp_extract_all is very useful, if we just reference the description.

### Why are the changes needed?
`regexp_extract_all` is less common.

### Does this PR introduce _any_ user-facing change?
'No'. `regexp_extract_all` was added in Spark 3.1.0 which isn't released yet.

### How was this patch tested?
Jenkins test.

Closes #31346 from beliefer/SPARK-24884-followup.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 13:52:51 -08:00
Max Gekk ac8307d75c [SPARK-34215][SQL] Keep tables cached after truncation
### What changes were proposed in this pull request?
Invoke `CatalogImpl.refreshTable()` instead of combination of `SessionCatalog.refreshTable()` + `uncacheQuery()`. This allows to clear cached table data while keeping the table cached.

### Why are the changes needed?
1. To improve user experience with Spark SQL
2. To be consistent to other commands, see https://github.com/apache/spark/pull/31206

### Does this PR introduce _any_ user-facing change?
Yes.

Before:
```scala
scala> sql("CREATE TABLE tbl (c0 int)")
res1: org.apache.spark.sql.DataFrame = []
scala> sql("INSERT INTO tbl SELECT 0")
res2: org.apache.spark.sql.DataFrame = []
scala> sql("CACHE TABLE tbl")
res3: org.apache.spark.sql.DataFrame = []
scala> sql("SELECT * FROM tbl").show(false)
+---+
|c0 |
+---+
|0  |
+---+
scala> spark.catalog.isCached("tbl")
res5: Boolean = true
scala> sql("TRUNCATE TABLE tbl")
res6: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.isCached("tbl")
res7: Boolean = false
```

After:
```scala
scala> sql("TRUNCATE TABLE tbl")
res6: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.isCached("tbl")
res7: Boolean = true
```

### How was this patch tested?
Added new test to `CachedTableSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31308 from MaxGekk/truncate-table-cached.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:36:44 +00:00
Angerszhuuuu dd88eff820 [SPARK-34241][SQL] For DDL command plan, we should define producedAttributes as it's outputSet
### What changes were proposed in this pull request?

When write test about command,  when `checkAnswer`,
Always got error as below
```
[info]   AttributeSet(partition#607) was not empty The analyzed logical plan has missing inputs:
[info]   ShowPartitionsCommand `ns`.`tbl`, [partition#607] (QueryTest.scala:224)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
```

For Command DDL plan, we can define  `producedAttributes` as it's `outputSet` and it's reasonable

### Why are the changes needed?
Add default   `producedAttributes` for Command LogicalPlan

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Not need

Closes #31342 from AngersZhuuuu/SPARK-34241.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:14:10 +00:00
Anton Okolnychyi 08679646fe [SPARK-34026][SQL] Inject repartition and sort nodes to satisfy required distribution and ordering
### What changes were proposed in this pull request?

This PR adds repartition and sort nodes to satisfy the required distribution and ordering introduced in SPARK-33779.

Note: This PR contains the final part of changes discussed in PR #29066.

### Why are the changes needed?

These changes are the next step as discussed in the [design doc](https://docs.google.com/document/d/1X0NsQSryvNmXBY9kcvfINeYyKC-AahZarUqg3nS1GQs/edit#) for SPARK-23889.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

This PR comes with a new test suite.

Closes #31083 from aokolnychyi/spark-34026.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:09:30 +00:00
yangjie01 8999e8805d [SPARK-34224][CORE][SQL][SS][DSTREAM][YARN][TEST][EXAMPLES] Ensure all resource opened by Source.fromXXX are closed
### What changes were proposed in this pull request?
Using a function like `.mkString` or `.getLines` directly on a `scala.io.Source` opened by `fromFile`, `fromURL`, `fromURI ` will leak the underlying file handle,  this pr use the `Utils.tryWithResource` method wrap the `BufferedSource` to ensure these `BufferedSource` closed.

### Why are the changes needed?
Avoid file handle leak.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31323 from LuciferYang/source-not-closed.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 19:06:37 +09:00
Wenchen Fan 8dee8a9b7c [SPARK-34227][SQL] WindowFunctionFrame should clear its states during preparation
### What changes were proposed in this pull request?

This PR fixed all `OffsetWindowFunctionFrameBase#prepare` implementations to reset the states, and also add more comments in `WindowFunctionFrame` classdoc to explain why we need to reset states during preparation: `WindowFunctionFrame` instances are reused to process multiple partitions.

### Why are the changes needed?

To fix a correctness bug caused by the new feature "window function with ignore nulls" in the master branch.

### Does this PR introduce _any_ user-facing change?

yes

### How was this patch tested?

new test

Closes #31325 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 08:50:14 +00:00
Yuanjian Li 0a1a029622 [SPARK-34235][SS] Make spark.sql.hive as a private package
### What changes were proposed in this pull request?
Follow the comment https://github.com/apache/spark/pull/31271#discussion_r562598983:

- Remove the API tag `Unstable` for `HiveSessionStateBuilder`
- Add document for spark.sql.hive package to emphasize it's a private package

### Why are the changes needed?
Follow the rule for a private package.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Doc change only.

Closes #31321 from xuanyuanking/SPARK-34185-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 17:13:11 +09:00
Warren Zhu 68b765e6b8 [SPARK-34232][CORE] Redact SparkListenerEnvironmentUpdate event in log
### What changes were proposed in this pull request?
Redact event SparkListenerEnvironmentUpdate in log when its processing time exceeded logSlowEventThreshold

### Why are the changes needed?
Credentials could be exposed when its processing time exceeded logSlowEventThreshold

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually tested

Closes #31335 from warrenzhu25/34232.

Authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 17:10:53 +09:00
Dongjoon Hyun 785d5822e5 [SPARK-34218][INFRA][FOLLOWUP] Fix Scala 2.13 profile typo in publish-snapshot
### What changes were proposed in this pull request?

This is a follow-up of #31311 and fixes a typo in Scala 2.13 profile section in `publish-snapshot` command.

### Why are the changes needed?

To fix snapshot publishing for Scala 2.13.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Manual.

Closes #31338 from dongjoon-hyun/SPARK-34218-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 15:11:07 +09:00
Dongjoon Hyun 7d09eac1cc [SPARK-34229][SQL] Avro should read decimal values with the file schema
### What changes were proposed in this pull request?

This PR aims to fix Avro data source to use the decimal precision and scale of file schema.

### Why are the changes needed?

The decimal value should be interpreted with its original precision and scale. Otherwise, it returns incorrect result like the following. The schema mismatch happens when we use `userSpecifiedSchema` or there are multiple files with inconsistent schema or HiveMetastore schema is updated by the user.
```scala
scala> sql("SELECT 3.14 a").write.format("avro").save("/tmp/avro")
scala> spark.read.schema("a DECIMAL(4, 3)").format("avro").load("/tmp/avro").show
+-----+
|    a|
+-----+
|0.314|
+-----+
```

### Does this PR introduce _any_ user-facing change?

Yes, this will return correct result.

### How was this patch tested?

Pass the CI with the newly added test case.

Closes #31329 from dongjoon-hyun/SPARK-34229.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 14:23:54 +09:00
Angerszhuuuu 7bd4165c11 [SPARK-32852][SQL][FOLLOW_UP] Add notice about keep hive version consistence when config hive jars location
### What changes were proposed in this pull request?
Add notice about keep hive version consistence when config hive jars location

With PR #29881, if we don't keep hive version consistence. we will got below error.
```
Builtin jars can only be used when hive execution version == hive metastore version. Execution: 2.3.8 != Metastore: 1.2.1. Specify a valid path to the correct hive jars using spark.sql.hive.metastore.jars or change spark.sql.hive.metastore.version to 2.3.8.
```

![image](https://user-images.githubusercontent.com/46485123/105795169-512d8380-5fc7-11eb-97c3-0259a0d2aa58.png)

### Why are the changes needed?
Make config doc detail

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Not need

Closes #31317 from AngersZhuuuu/SPARK-32852-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 13:40:20 +09:00
Kent Yao b3915ddd91 [SPARK-34223][SQL] FIX NPE for static partition with null in InsertIntoHadoopFsRelationCommand
### What changes were proposed in this pull request?

with a simple case, the null will be passed to InsertIntoHadoopFsRelationCommand blindly, we should avoid the npe
```scala
 test("NPE") {
    withTable("t") {
      sql(s"CREATE TABLE t(i STRING, c string) USING $format PARTITIONED BY (c)")
      sql("INSERT OVERWRITE t PARTITION (c=null) VALUES ('1')")
      checkAnswer(spark.table("t"), Row("1", null))
    }
  }
```
```logtalk
java.lang.NullPointerException
	at scala.collection.immutable.StringOps$.length(StringOps.scala:51)
	at scala.collection.immutable.StringOps.length(StringOps.scala:51)
	at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:35)
	at scala.collection.IndexedSeqOptimized.foreach
	at scala.collection.immutable.StringOps.foreach(StringOps.scala:33)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.escapePathName(ExternalCatalogUtils.scala:69)
	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.orig-s0.0000030000-r30676-expand-or-complete(InsertIntoHadoopFsRelationCommand.scala:231)
```

### Why are the changes needed?

a bug fix
### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

new tests

Closes #31320 from yaooqinn/SPARK-34223.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 12:05:58 +08:00
Ruifeng Zheng cb37c962be [SPARK-31768][ML][FOLLOWUP] add getMetrics in Evaluators: cleanup
### What changes were proposed in this pull request?
1, make `silhouette` a method;
2, change return type of `setDistanceMeasure` to `this.type`;

### Why are the changes needed?
see comments in https://github.com/apache/spark/pull/28590

### Does this PR introduce _any_ user-facing change?
No, 3.1 has not been released

### How was this patch tested?
existing testsuites

Closes #31334 from zhengruifeng/31768-followup.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Weichen Xu <weichen.xu@databricks.com>
2021-01-26 11:57:28 +08:00
Kent Yao d1177b5230 [SPARK-34192][SQL] Move char padding to write side and remove length check on read side too
### What changes were proposed in this pull request?

On the read-side, the char length check and padding bring issues to CBO and predicate pushdown and other issues to the catalyst.

This PR reverts 6da5cdf1db  that added read side length check) so that we only do length check for the write side, and data sources/vendors are responsible to enforce the char/varchar constraints for data import operations like ADD PARTITION. It doesn't make sense for Spark to report errors on the read-side if the data is already dirty.

This PR also moves the char padding to the write-side, so that it 1) avoids read side issues like CBO and filter pushdown. 2) the data source can preserve char type semantic better even if it's read by systems other than Spark.

### Why are the changes needed?

fix perf regression when tables have char/varchar type columns

closes #31278
### Does this PR introduce _any_ user-facing change?

yes, spark will not raise error for oversized char/varchar values in read side
### How was this patch tested?

modified ut

the dropped read side benchmark
```
================================================================================================
Char Varchar Read Side Perf w/o Tailing Spaces
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 20:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 20                         1564           1573           9         63.9          15.6       1.0X
read char with length 20                           1532           1551          18         65.3          15.3       1.0X
read varchar with length 20                        1520           1531          13         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 40:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 40                         1573           1613          41         63.6          15.7       1.0X
read char with length 40                           1575           1577           2         63.5          15.7       1.0X
read varchar with length 40                        1568           1576          11         63.8          15.7       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 60:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 60                         1526           1540          23         65.5          15.3       1.0X
read char with length 60                           1514           1539          23         66.0          15.1       1.0X
read varchar with length 60                        1486           1497          10         67.3          14.9       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 80:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 80                         1531           1542          19         65.3          15.3       1.0X
read char with length 80                           1514           1529          15         66.0          15.1       1.0X
read varchar with length 80                        1524           1565          42         65.6          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 100:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 100                        1597           1623          25         62.6          16.0       1.0X
read char with length 100                          1499           1512          16         66.7          15.0       1.1X
read varchar with length 100                       1517           1524           8         65.9          15.2       1.1X

================================================================================================
Char Varchar Read Side Perf w/ Tailing Spaces
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 20:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 20                         1524           1526           1         65.6          15.2       1.0X
read char with length 20                           1532           1537           9         65.3          15.3       1.0X
read varchar with length 20                        1520           1532          15         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 40:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 40                         1556           1580          32         64.3          15.6       1.0X
read char with length 40                           1600           1611          17         62.5          16.0       1.0X
read varchar with length 40                        1648           1716          88         60.7          16.5       0.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 60:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 60                         1504           1524          20         66.5          15.0       1.0X
read char with length 60                           1509           1512           3         66.2          15.1       1.0X
read varchar with length 60                        1519           1535          21         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 80:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 80                         1640           1652          17         61.0          16.4       1.0X
read char with length 80                           1625           1666          35         61.5          16.3       1.0X
read varchar with length 80                        1590           1605          13         62.9          15.9       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 100:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 100                        1622           1628           5         61.6          16.2       1.0X
read char with length 100                          1614           1646          30         62.0          16.1       1.0X
read varchar with length 100                       1594           1606          11         62.7          15.9       1.0X
```

Closes #31281 from yaooqinn/SPARK-34192.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 02:08:35 +08:00
Dongjoon Hyun bdf71be0ff [SPARK-34218][INFRA] Add Scala 2.13 packaging and publishing
### What changes were proposed in this pull request?

This PR aims to add `Scala 2.13` packaging and publishing.

### Why are the changes needed?

To support Scala 2.13 officially in Apache Spark 3.2.0, we need to publish the artifacts.

### Does this PR introduce _any_ user-facing change?

Yes, this will provide additional artifacts.

### How was this patch tested?

Manual.

Closes #31311 from dongjoon-hyun/SPARK-34218.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-25 09:01:26 -08:00
Max Gekk bfc0235013 [SPARK-34203][SQL] Convert null partition values to __HIVE_DEFAULT_PARTITION__ in v1 In-Memory catalog
### What changes were proposed in this pull request?
In the PR, I propose to convert `null` partition values to `"__HIVE_DEFAULT_PARTITION__"` before storing in the `In-Memory` catalog internally. Currently, the `In-Memory` catalog maintains null partitions as `"__HIVE_DEFAULT_PARTITION__"` in file system but as `null` values in memory that could cause some issues like in SPARK-34203.

### Why are the changes needed?
`InMemoryCatalog` stores partitions in the file system in the Hive compatible form, for instance, it converts the `null` partition value to `"__HIVE_DEFAULT_PARTITION__"` but at the same time it keeps null as is internally. That causes an issue demonstrated by the example below:
```
$ ./bin/spark-shell -c spark.sql.catalogImplementation=in-memory
```
```scala
scala> spark.conf.get("spark.sql.catalogImplementation")
res0: String = in-memory

scala> sql("CREATE TABLE tbl (col1 INT, p1 STRING) USING parquet PARTITIONED BY (p1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("INSERT OVERWRITE TABLE tbl VALUES (0, null)")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE tbl DROP PARTITION (p1 = null)")
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionsException: The following partitions not found in table 'tbl' database 'default':
Map(p1 -> null)
  at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.dropPartitions(InMemoryCatalog.scala:440)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `ALTER TABLE .. DROP PARTITION` can drop the `null` partition in `In-Memory` catalog:
```scala
scala> spark.table("tbl").show(false)
+----+----+
|col1|p1  |
+----+----+
|0   |null|
+----+----+

scala> sql("ALTER TABLE tbl DROP PARTITION (p1 = null)")
res4: org.apache.spark.sql.DataFrame = []

scala> spark.table("tbl").show(false)
+----+---+
|col1|p1 |
+----+---+
+----+---+
```

### How was this patch tested?
Added new test to `AlterTableDropPartitionSuiteBase`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #31322 from MaxGekk/insert-overwrite-null-part.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 15:27:20 +00:00
Dereck Li 096b15fa12 [SPARK-34607][SQL][FOLLOWUP] Change Option[LogicalRelation] to LogicalRelation
### What changes were proposed in this pull request?
optimize: change Option[LogicalRelation] to LogicalRelation

### Why are the changes needed?
simplify code

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
Existed unit test.

Closes #31315 from monkeyboy123/spark-34067-follow-up.

Authored-by: Dereck Li <monkeyboy.ljh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 13:34:07 +00:00
Max Gekk 6fe5a8a2ae [SPARK-34197][SQL] SessionCatalog.refreshTable() should not invalidate the relation cache for temporary views
### What changes were proposed in this pull request?
Check the name passed to `SessionCatalog.refreshTable`, and if it belongs to a temporary view, do not invalidate the relation cache.

### Why are the changes needed?
When `SessionCatalog.refreshTable` refreshes a temporary or global temporary view, it should not invalidate an entry in the relation cache associated to a table with the same name.

### Does this PR introduce _any_ user-facing change?
Should not. The change might improve performance slightly.

### How was this patch tested?
By running new UT:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *SessionCatalogSuite"
```

Closes #31265 from MaxGekk/fix-session-catalog-refresh-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 07:37:24 +00:00
yliou 512cacf7c6 [SPARK-33726][SQL] Fix for Duplicate field names during Aggregation
### What changes were proposed in this pull request?
The `RowBasedKeyValueBatch` has two different implementations depending on whether the aggregation key and value uses only fixed length data types (`FixedLengthRowBasedKeyValueBatch`) or not (`VariableLengthRowBasedKeyValueBatch`).

Before this PR the decision about the used implementation was based on by accessing the schema fields by their name.
But if two fields has the same name and one with variable length and the other with fixed length type (and all the other fields are with fixed length types) a bad decision could be made.

When `FixedLengthRowBasedKeyValueBatch` is chosen but there is a variable length field then an aggregation function could calculate with invalid values. This case is illustrated by the example used in the unit test:

`with T as (select id as a, -id as x from range(3)),
        U as (select id as b, cast(id as string) as x from range(3))
select T.x, U.x, min(a) as ma, min(b) as mb from T join U on a=b group by U.x, T.x`
where the 'x' column in the left side of the join is a Long but on the right side is a String.

### Why are the changes needed?
Fixes the issue where duplicate field name aggregation has null values in the dataframe.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added UT, tested manually on spark shell.

Closes #30788 from yliou/SPARK-33726.

Authored-by: yliou <yliou@berkeley.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 06:53:26 +00:00
Erik Krogen 9371ea8c7b [SPARK-34133][AVRO] Respect case sensitivity when performing Catalyst-to-Avro field matching
### What changes were proposed in this pull request?
Make the field name matching between Avro and Catalyst schemas, on both the reader and writer paths, respect the global SQL settings for case sensitivity (i.e. case-insensitive by default). `AvroSerializer` and `AvroDeserializer` share a common utility in `AvroUtils` to search for an Avro field to match a given Catalyst field.

### Why are the changes needed?
Spark SQL is normally case-insensitive (by default), but currently when `AvroSerializer` and `AvroDeserializer` perform matching between Catalyst schemas and Avro schemas, the matching is done in a case-sensitive manner. So for example the following will fail:
```scala
      val avroSchema =
        """
          |{
          |  "type" : "record",
          |  "name" : "test_schema",
          |  "fields" : [
          |    {"name": "foo", "type": "int"},
          |    {"name": "BAR", "type": "int"}
          |  ]
          |}
      """.stripMargin
      val df = Seq((1, 3), (2, 4)).toDF("FOO", "bar")

      df.write.option("avroSchema", avroSchema).format("avro").save(savePath)
```

The same is true on the read path, if we assume `testAvro` has been written using the schema above, the below will fail to match the fields:
```scala
df.read.schema(new StructType().add("FOO", IntegerType).add("bar", IntegerType))
  .format("avro").load(testAvro)
```

### Does this PR introduce _any_ user-facing change?
When reading Avro data, or writing Avro data using the `avroSchema` option, field matching will be performed with case sensitivity respecting the global SQL settings.

### How was this patch tested?
New tests added to `AvroSuite` to validate the case sensitivity logic in an end-to-end manner through the SQL engine.

Closes #31201 from xkrogen/xkrogen-SPARK-34133-avro-serde-casesensitivity-errormessages.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 04:54:41 +00:00
Kousuke Saruta 144ee9eb30 [SPARK-34186][SQL][TESTS] Fix DockerJDBCIntegrationSuites to reflect the change of SPARK-33888
### What changes were proposed in this pull request?

This PR fixes the following integration suites to reflect the change of SPARK-33888.

* PostgresIntegrationSuite
* MySQLIntegrationSuite
* MsSqlServerIntegrationSuite
* DB2IntegrationSuite

### Why are the changes needed?

Those suites doesn't pass currently in `master` branch.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Those suites pass.

Closes #31274 from sarutak/fix-integration-suites.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-24 18:44:14 -08:00
Peter Toth 98ec6c27e3 [SPARK-34147][SQL][TEST] Keep table partitioning in TPCDSQueryBenchmak when CBO is enabled
### What changes were proposed in this pull request?
This PR keeps partitioning of input tables in TPCDSQueryBenchmark when `--cbo` option is enabled.

https://github.com/apache/spark/pull/31011 introduced the `--cbo` option but unfortunately in that mode the table partitioning of the input data is lost. This means that the results of CBO mode is very different to non CBO mode, one example is that Dynamic Partition Pruning doesn't kick in in CBO mode.

### Why are the changes needed?
To monitor performance changed with CBO enabled.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Manually checked.

Closes #31218 from peter-toth/SPARK-34147-keep-partitioning-in-tpcdsquerybenchmark.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:42:47 +09:00
Yuanjian Li 59cbacaddf [SPARK-34185][DOCS] Review and fix issues in API docs
### What changes were proposed in this pull request?
Compare the 3.1.1 API doc with the latest release version 3.0.1. Fix the following issues:
- Add missing `Since` annotation for new APIs
- Remove the leaking class/object in API doc

### Why are the changes needed?
Fix the issues in the Spark 3.1.1 release API docs.

### Does this PR introduce _any_ user-facing change?
Yes, API doc changes.

### How was this patch tested?
Manually test.

Closes #31271 from xuanyuanking/SPARK-34185.

Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:38:20 +09:00