Commit graph

30616 commits

Author SHA1 Message Date
Gengliang Wang ac1c6aa45c [SPARK-35987][SQL] The ANSI flags of Sum and Avg should be kept after being copied
### What changes were proposed in this pull request?

Make the ANSI flag part of expressions `Sum` and `Average`'s parameter list, instead of fetching it from the sessional SQLConf.

### Why are the changes needed?

For Views, it is important to show consistent results even the ANSI configuration is different in the running session. This is why many expressions like 'Add'/'Divide' making the ANSI flag part of its case class parameter list.

We should make it consistent for the expressions `Sum` and `Average`

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

Yes, the `Sum` and `Average` inside a View always behaves the same, independent of the ANSI model SQL configuration in the current session.

### How was this patch tested?

Existing UT

Closes #33186 from gengliangwang/sumAndAvg.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 51103cdcdd)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-05 12:34:39 +08:00
Kevin Su 873f6b9d97 [SPARK-36007][INFRA] Failed to run benchmark in GA
When I'm running the benchmark in GA, I met the below error.

https://github.com/pingsutw/spark/runs/2867617238?check_suite_focus=true
```
java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.j
ava:1692)java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
21/06/20 07:40:02 ERROR SparkContext: Error initializing SparkContext.java.lang.AssertionError: assertion failed:
spark.test.home is not set! at scala.Predef$.assert(Predef.scala:223) at org.apache.spark.deploy.worker.Worker.<init>
(Worker.scala:148) at org.apache.spark.deploy.worker.Worker$.startRpcEnvAndEndpoint(Worker.scala:954) at
org.apache.spark.deploy.LocalSparkCluster.$anonfun$start$2(LocalSparkCluster.scala:68) at
org.apache.spark.deploy.LocalSparkCluster.$anonfun$start$2$adapted(LocalSparkCluster.scala:65) at
scala.collection.immutable.Range.foreach(Range.scala:158) at
org.apache.spark.deploy.LocalSparkCluster.start(LocalSparkCluster.scala:65) at
org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$createTaskScheduler(SparkContext.scala:2954) at
org.apache.spark.SparkContext.<init>(SparkContext.scala:559) at org.apache.spark.SparkContext.<init>
(SparkContext.scala:137) at
org.apache.spark.serializer.KryoSerializerBenchmark$.createSparkContext(KryoSerializerBenchmark.scala:86) at
org.apache.spark.serializer.KryoSerializerBenchmark$.sc$lzycompute$1(KryoSerializerBenchmark.scala:58) at
org.apache.spark.serializer.KryoSerializerBenchmark$.sc$1(KryoSerializerBenchmark.scala:58) at
org.apache.spark.serializer.KryoSerializerBenchmark$.$anonfun$run$3(KryoSerializerBenchmark.scala:63)
```

Set `spark.test.home` in the benchmark workflow.

No

Rerun the benchmark in my fork.
https://github.com/pingsutw/spark/actions/runs/996067851

Closes #33203 from pingsutw/SPARK-36007.

Lead-authored-by: Kevin Su <pingsutw@apache.org>
Co-authored-by: Kevin Su <pingsutw@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 11fcbc73cb)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-05 09:18:03 +09:00
Tomas Pereira de Vasconcelos e8991266c8 [SPARK-35986][PYSPARK] Fix type hint for RDD.histogram's buckets
Fix the type hint for `pyspark.rdd .RDD.histogram`'s `buckets` argument

The current type hint is incomplete.
![image](https://user-images.githubusercontent.com/17701527/124248180-df7fd580-db22-11eb-8391-ba0bb51d689b.png)
From `pyspark.rdd .RDD.histogram`'s source:
```python
if isinstance(buckets, int):
    ...
elif isinstance(buckets, (list, tuple)):
    ...
else:
    raise TypeError("buckets should be a list or tuple or number(int or long)")
```

Fixed the warning displayed above.

Fixed warning above with this change.

Closes #33185 from tpvasconcelos/master.

Authored-by: Tomas Pereira de Vasconcelos <tomasvasconcelos1@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 495d234c6e)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-04 10:24:55 +09:00
Wenchen Fan ec84982191 [SPARK-35940][SQL] Refactor EquivalentExpressions to make it more efficient
### What changes were proposed in this pull request?

This PR uses 2 ideas to make `EquivalentExpressions` more efficient:
1. do not keep all the equivalent expressions, we only need a count
2. track the "height" of common subexpressions, to quickly do child-parent sort, and filter out non-child expressions in `addCommonExprs`

This PR also fixes several small bugs (exposed by the refactoring), please see PR comments.

### Why are the changes needed?

code cleanup and small perf improvement

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

no

### How was this patch tested?

existing tests

Closes #33142 from cloud-fan/codegen.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit e6ce220690)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-03 08:28:59 -07:00
Takuya UESHIN 8bc54c2d6d [SPARK-35995][INFRA][3.2][FOLLOWUP] Fix the reference branches and PR builder
### What changes were proposed in this pull request?

Fixes the reference branches and PR builder.

### Why are the changes needed?

Currently the build and tests are on `master` branch because the reference branch is always `master`.

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

No.

### How was this patch tested?

N/A

Closes #33197 from ueshin/issues/SPARK-35995/branch-3.2_build.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 14:47:45 -07:00
Takuya UESHIN fcc9e66c9b [SPARK-35981][PYTHON][TEST][3.2] Use check_exact=False to loosen the check precision
### What changes were proposed in this pull request?

This is a cherry-pick of #33179.

We should use `check_exact=False` because the value check in `StatsTest.test_cov_corr_meta` is too strict.

### Why are the changes needed?

In some environment, the precision could be different in pandas' `DataFrame.corr` function and the test `StatsTest.test_cov_corr_meta` fails.

```
AssertionError: DataFrame.iloc[:, 0] (column name="a") are different
DataFrame.iloc[:, 0] (column name="a") values are different (14.28571 %)
[index]: [a, b, c, d, e, f, g]
[left]:  [1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 0.0]
[right]: [1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 4.807406715958909e-17]
```

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

No.

### How was this patch tested?

Modified tests should still pass.

Closes #33193 from ueshin/issuse/SPARK-35981/3.2/corr.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 14:08:50 -07:00
Kousuke Saruta 9b0971c40e [SPARK-35990][BUILD] Remove avro-sbt plugin dependency
### What changes were proposed in this pull request?

This PR removes sbt-avro plugin dependency.
In the current master, Build with SBT depends on the plugin but it seems never used.
Originally, the plugin was introduced for `flume-sink` in SPARK-1729 (#807) but `flume-sink` is no longer in Spark repository.

After SBT was upgraded to 1.x in SPARK-21708 (#29286), `avroGenerate` part was introduced in `object SQL` in `SparkBuild.scala`.
It's confusable but I understand `Test / avroGenerate := (Compile / avroGenerate).value` is for suppressing sbt-avro for `sql` sub-module.
In fact, Test/compile will fail if `Test / avroGenerate :=(Compile / avroGenerate).value` is commented out.

`sql` sub-module contains `parquet-compat.avpr` and `parquet-compat.avdl` but according to `sql/core/src/test/README.md`, they are intended to be handled by `gen-avro.sh`.

Also, in terms of Maven build, there seems to be no definition to handle `*.avpr` or `*.avdl`.

Based on the above, I think we can remove `sbt-avro`.

### Why are the changes needed?

If `sbt-avro` is really no longer used, it's confusable that `sbt-avro` related configurations are in `SparkBuild.scala`.

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

No.

### How was this patch tested?

GA.

Closes #33190 from sarutak/remove-avro-from-sbt.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 6c4616b2ac)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 11:01:12 -07:00
Liang-Chi Hsieh 0db6f3a844 [SPARK-35785][SS][FOLLOWUP] Ignore concurrent update and cleanup test
### What changes were proposed in this pull request?

This patch ignores the test "ensure that concurrent update and cleanup consistent versions" in #32933. The test is currently flaky and we will address it later.

### Why are the changes needed?

Unblock other developments.

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

No

### How was this patch tested?

Existing tests.

Closes #33195 from viirya/ignore-rocksdb-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit a6e00ee9d7)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 10:58:59 -07:00
Dongjoon Hyun 3c36c6f997 [SPARK-35995][INFRA][3.2] Enable GitHub Action build_and_test on branch-3.2
### What changes were proposed in this pull request?

Like branch-3.1, this PR aims to update GitHub Action `build_and_test` in branch-3.2.

### Why are the changes needed?

Currently, GitHub Action on branch-3.2 is working.
- https://github.com/apache/spark/commits/branch-3.2

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

No.

### How was this patch tested?

N/A

Closes #33194 from dongjoon-hyun/SPARK-35995.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 10:56:42 -07:00
Dongjoon Hyun d7990943c3 [SPARK-35992][BUILD] Upgrade ORC to 1.6.9
### What changes were proposed in this pull request?

This PR aims to upgrade Apache ORC to 1.6.9.

### Why are the changes needed?

This is required to bring ORC-804 in order to fix ORC encryption masking bug.

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

No. This is not released yet.

### How was this patch tested?

Pass the newly added test case.

Closes #33189 from dongjoon-hyun/SPARK-35992.

Lead-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit c55b9fd1e0)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 09:50:00 -07:00
Yuanjian Li ae8f91734d [SPARK-35785][SS][3.2] Cleanup support for RocksDB instance
### What changes were proposed in this pull request?
Add the functionality of cleaning up files of old versions for the RocksDB instance and RocksDBFileManager.

### Why are the changes needed?
Part of the implementation of RocksDB state store.

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

### How was this patch tested?
New UT added.

Closes #33184 from xuanyuanking/branch-3.2.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-02 17:12:00 +09:00
Wenchen Fan c1d8178817 [SPARK-35968][SQL] Make sure partitions are not too small in AQE partition coalescing
### What changes were proposed in this pull request?

By default, AQE will set `COALESCE_PARTITIONS_MIN_PARTITION_NUM` to the spark default parallelism, which is usually quite big. This is to keep the parallelism on par with non-AQE, to avoid perf regressions.

However, this usually leads to many small/empty partitions, and hurts performance (although not worse than non-AQE). Users usually blindly set `COALESCE_PARTITIONS_MIN_PARTITION_NUM` to 1, which makes this config quite useless.

This PR adds a new config to set the min partition size, to avoid too small partitions after coalescing. By default, Spark will not respect the target size, and only respect this min partition size, to maximize the parallelism and avoid perf regression in AQE. This PR also adds a bool config to respect the target size when coalescing partitions, and it's recommended to set it to get better overall performance. This PR also deprecates the `COALESCE_PARTITIONS_MIN_PARTITION_NUM` config.

### Why are the changes needed?

AQE is default on now, we should make the perf better in the default case.

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

yes, a new config.

### How was this patch tested?

new tests

Closes #33172 from cloud-fan/aqe2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 0c9c8ff569)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-02 16:07:46 +08:00
Dongjoon Hyun 79a6e00b76 [SPARK-35825][INFRA][FOLLOWUP] Increase it in build/mvn script
### What changes were proposed in this pull request?

This is a follow up of https://github.com/apache/spark/pull/32961.

This PR additionally sets the stack size in `build/mvn`.

### Why are the changes needed?

We are still hitting `StackOverflowError` in Jenkins.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2/3064/console
```
[INFO] compiling 166 Scala sources and 19 Java sources to /home/jenkins/workspace/spark-master-test-maven-hadoop-3.2/sql/catalyst/target/scala-2.12/classes ...
[ERROR] ## Exception when compiling 480 sources to /home/jenkins/workspace/spark-master-test-maven-hadoop-3.2/sql/catalyst/target/scala-2.12/classes
java.lang.StackOverflowError
```

This PR increases the JVM of `mvn` instead of the plugin.

```
$ MAVEN_OPTS="-XX:+PrintFlagsFinal" build/mvn clean | grep 'intx ThreadStackSize'
     intx ThreadStackSize                           = 2048                                {pd product}

$ MAVEN_OPTS="-Xss128m -XX:+PrintFlagsFinal" build/mvn clean | grep 'intx ThreadStackSize'
     intx ThreadStackSize                          := 131072                              {pd product}
```

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

No.

### How was this patch tested?

N/A

Closes #33180 from dongjoon-hyun/SPARK-35825.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-01 22:24:48 -07:00
Karen Feng 1fda011d71 [SPARK-35955][SQL] Check for overflow in Average in ANSI mode
### What changes were proposed in this pull request?

Fixes decimal overflow issues for decimal average in ANSI mode, so that overflows throw an exception rather than returning null.

### Why are the changes needed?

Query:

```
scala> import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions._

scala> spark.conf.set("spark.sql.ansi.enabled", true)

scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(mean("decNum"))
df2: org.apache.spark.sql.DataFrame = [avg(decNum): decimal(38,22)]

scala> df2.show(40,false)
```

Before:
```
+-----------+
|avg(decNum)|
+-----------+
|null       |
+-----------+
```

After:
```
21/07/01 19:48:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 24)
java.lang.ArithmeticException: Overflow in sum of decimals.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.overflowInSumOfDecimalError(QueryExecutionErrors.scala:162)
	at org.apache.spark.sql.errors.QueryExecutionErrors.overflowInSumOfDecimalError(QueryExecutionErrors.scala)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:499)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:502)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

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

No

### How was this patch tested?

Unit test

Closes #33177 from karenfeng/SPARK-35955.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-02 12:41:24 +08:00
Rahul Mahadev 47485a3c2d [SPARK-35897][SS] Support user defined initial state with flatMapGroupsWithState in Structured Streaming
### What changes were proposed in this pull request?
This PR aims to add support for specifying a user defined initial state for arbitrary structured streaming stateful processing using [flat]MapGroupsWithState operator.

### Why are the changes needed?
Users can load previous state of their stateful processing as an initial state instead of redoing the entire processing once again.

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

Yes this PR introduces new API
```
  def mapGroupsWithState[S: Encoder, U: Encoder](
      timeoutConf: GroupStateTimeout,
      initialState: KeyValueGroupedDataset[K, S])(
      func: (K, Iterator[V], GroupState[S]) => U): Dataset[U]

  def flatMapGroupsWithState[S: Encoder, U: Encoder](
      outputMode: OutputMode,
      timeoutConf: GroupStateTimeout,
      initialState: KeyValueGroupedDataset[K, S])(
      func: (K, Iterator[V], GroupState[S]) => Iterator[U])

```

### How was this patch tested?

Through unit tests in FlatMapGroupsWithStateSuite

Closes #33093 from rahulsmahadev/flatMapGroupsWithState.

Authored-by: Rahul Mahadev <rahul.mahadev@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-02 11:53:17 +08:00
Xinrong Meng 95d94948c5 [SPARK-35339][PYTHON] Improve unit tests for data-type-based basic operations
### What changes were proposed in this pull request?

Improve unit tests for data-type-based basic operations by:
- removing redundant test cases
- adding `astype` test for ExtensionDtypes

### Why are the changes needed?

Some test cases for basic operations are duplicated after introducing data-type-based basic operations. The PR is proposed to remove redundant test cases.
`astype` is not tested for ExtensionDtypes, which will be adjusted in this PR as well.

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

No.

### How was this patch tested?

Unit tests.

Closes #33095 from xinrong-databricks/datatypeops_test.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-07-01 17:37:32 -07:00
Anton Okolnychyi fceabe2372 [SPARK-35779][SQL] Dynamic filtering for Data Source V2
### What changes were proposed in this pull request?

This PR implemented the proposal per [design doc](https://docs.google.com/document/d/1RfFn2e9o_1uHJ8jFGsSakp-BZMizX1uRrJSybMe2a6M) for SPARK-35779.

### Why are the changes needed?

Spark supports dynamic partition filtering that enables reusing parts of the query to skip unnecessary partitions in the larger table during joins. This optimization has proven to be beneficial for star-schema queries which are common in the industry. Unfortunately, dynamic pruning is currently limited to partition pruning during joins and is only supported for built-in v1 sources. As more and more Spark users migrate to Data Source V2, it is important to generalize dynamic filtering and expose it to all v2 connectors.

Please, see the design doc for more information on this effort.

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

Yes, this PR adds a new optional mix-in interface for `Scan` in Data Source V2.

### How was this patch tested?

This PR comes with tests.

Closes #32921 from aokolnychyi/dynamic-filtering-wip.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-01 17:00:12 -07:00
Gengliang Wang a643076d4e [SPARK-35975][SQL] New configuration spark.sql.timestampType for the default timestamp type
### What changes were proposed in this pull request?

Add a new configuration `spark.sql.timestampType`, which configures the default timestamp type of Spark SQL, including SQL DDL and Cast clause. Setting the configuration as `TIMESTAMP_NTZ` will use `TIMESTAMP WITHOUT TIME ZONE` as the default type while putting it as `TIMESTAMP_LTZ` will use `TIMESTAMP WITH LOCAL TIME ZONE`.

The default value of the new configuration is TIMESTAMP_LTZ, which is consistent with previous Spark releases.

### Why are the changes needed?

A new configuration for switching the default timestamp type as timestamp without time zone.

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

No, it's a new feature.

### How was this patch tested?

Unit test

Closes #33176 from gengliangwang/newTsTypeConf.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-01 23:25:18 +03:00
SaurabhChawla ca1217667c [SPARK-35756][SQL] unionByName supports struct having same col names but different sequence
### What changes were proposed in this pull request?

unionByName does not supports struct having same col names but different sequence
```
val df1 = Seq((1, Struct1(1, 2))).toDF("a", "b")
val df2 = Seq((1, Struct2(1, 2))).toDF("a", "b")
val unionDF = df1.unionByName(df2)
```
it gives the exception

`org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. struct<c2:int,c1:int> <> struct<c1:int,c2:int> at the second column of the second table; 'Union false, false :- LocalRelation [_1#38, _2#39] +- LocalRelation _1#45, _2#46`

In this case the col names are same so this unionByName should have the support to check within in the Struct if col names are same it should not throw this exception and works.

after fix we are getting the result

```
val unionDF = df1.unionByName(df2)
scala>  unionDF.show
+---+------+
|  a|     b|
+---+------+
|  1|{1, 2}|
|  1|{2, 1}|
+---+------+

```

### Why are the changes needed?
As per unionByName functionality based on name, does the union. In the case of struct this scenario was missing where all the columns  names are same but sequence is different,  so added this functionality.

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

### How was this patch tested?
Added the unit test and also done the testing through spark shell

Closes #32972 from SaurabhChawla100/SPARK-35756.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 17:37:09 +00:00
Kent Yao 94c1e3c38c [SPARK-35969][K8S] Make the pod prefix more readable and tallied with K8S DNS Label Names
### What changes were proposed in this pull request?

By default, the executor pod prefix is generated by the app name. It handles characters that match [^a-z0-9\\-] differently. The '.' and all whitespaces will be converted to '-', but other ones to empty string. Especially,  characters like '_', '|' are commonly used as a word separator in many languages.

According to the K8S DNS Label Names, see https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#dns-label-names, we can convert all special characters to `-`.

 
For example,

```
scala> "xyz_abc_i_am_a_app_name_w/_some_abbrs".replaceAll("[^a-z0-9\\-]", "-").replaceAll("-+", "-")
res11: String = xyz-abc-i-am-a-app-name-w-some-abbrs

scala> "xyz_abc_i_am_a_app_name_w/_some_abbrs".replaceAll("\\s+", "-").replaceAll("\\.", "-").replaceAll("[^a-z0-9\\-]", "").replaceAll("-+", "-")
res12: String = xyzabciamaappnamewsomeabbrs
```

```scala
scala> "time.is%the¥most$valuable_——————thing,it's about time.".replaceAll("[^a-z0-9\\-]", "-").replaceAll("-+", "-")
res9: String = time-is-the-most-valuable-thing-it-s-about-time-

scala> "time.is%the¥most$valuable_——————thing,it's about time.".replaceAll("\\s+", "-").replaceAll("\\.", "-").replaceAll("[^a-z0-9\\-]", "").replaceAll("-+", "-")
res10: String = time-isthemostvaluablethingits-about-time-

```

### Why are the changes needed?

For better UX

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

yes, the executor pod name might look better
### How was this patch tested?

add new ones

Closes #33171 from yaooqinn/SPARK-35969.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-01 08:15:00 -07:00
Gengliang Wang 3acc4b973b [SPARK-35971][SQL] Rename the type name of TimestampNTZType as "timestamp_ntz"
### What changes were proposed in this pull request?

Rename the type name string of TimestampNTZType from "timestamp without time zone" to "timestamp_ntz".

### Why are the changes needed?

This is to make the column header shorter and simpler.
Snowflake and Flink uses similar approach:
https://docs.snowflake.com/en/sql-reference/data-types-datetime.html
https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/table/concepts/timezone/

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit tests

Closes #33173 from gengliangwang/reviseTypeName.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-01 20:50:19 +08:00
Linhong Liu 3c683434fa [SPARK-35686][SQL] Not allow using auto-generated alias when creating view
### What changes were proposed in this pull request?
As described in  #32831, Spark has compatible issues when querying a view created by an
older version. The root cause is that Spark changed the auto-generated alias name. To avoid
this in the future, we could ask the user to specify explicit column names when creating
a view.

### Why are the changes needed?
Avoid compatible issue when querying a view

### Does this PR introduce _any_ user-facing change?
Yes. User will get error when running query below after this change
```
CREATE OR REPLACE VIEW v AS SELECT CAST(t.a AS INT), to_date(t.b, 'yyyyMMdd') FROM t
```

### How was this patch tested?
not yet

Closes #32832 from linhongliu-db/SPARK-35686-no-auto-alias.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 12:47:38 +00:00
Kent Yao 6699f76fe2 [SPARK-35966][SQL] Port HIVE-17952: Fix license headers to avoid dangling javadoc warnings
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
  8. If you want to add or modify an error message, please read the guideline first:
     https://spark.apache.org/error-message-guidelines.html
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->

Port HIVE-17952: Fix license headers to avoid dangling javadoc warnings

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
Fix license headers

### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
no

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
pass rat check

Closes #33169 from yaooqinn/SPARK-35966.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-01 18:22:04 +08:00
Cheng Su 3c3193c0fc [SPARK-35965][DOCS] Add doc for ORC nested column vectorized reader
### What changes were proposed in this pull request?

In https://issues.apache.org/jira/browse/SPARK-34862, we added support for ORC nested column vectorized reader, and it is disabled by default for now. So we would like to add the user-facing documentation for it, and user can opt-in to use it if they want.

### Why are the changes needed?

To make user be aware of the feature, and let them know the instruction to use the feature.

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

Yes, the documentation itself.

### How was this patch tested?

Manually check generated documentation as below.

<img width="1153" alt="Screen Shot 2021-07-01 at 12 19 40 AM" src="https://user-images.githubusercontent.com/4629931/124083422-b0724280-da02-11eb-93aa-a25d118ba56e.png">

<img width="1147" alt="Screen Shot 2021-07-01 at 12 19 52 AM" src="https://user-images.githubusercontent.com/4629931/124083442-b5cf8d00-da02-11eb-899f-827d55b8558d.png">

Closes #33168 from c21/orc-doc.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-01 19:01:35 +09:00
Linhong Liu 0c34b96541 [SPARK-35685][SQL] Prompt recreating the view when there is an incompatible schema issue
### What changes were proposed in this pull request?
If the user creates a view in 2.4 and reads it in 3.1/3.2, there will be an incompatible schema issue.
So this PR adds a view ddl in the error message to prompt the user recreating the view to fix the
incompatible issue.
For example:
```sql
-- create view in 2.4
CREATE TABLE IF NOT EXISTS t USING parquet AS SELECT '1' as a, '20210420' as b"
CREATE OR REPLACE VIEW v AS SELECT CAST(t.a AS INT), to_date(t.b, 'yyyyMMdd') FROM t
-- select view in master
SELECT * FROM v
```
Then we will get below error:
```
cannot resolve '`to_date(spark_catalog.default.t.b, 'yyyyMMdd')`' given input columns: [a, to_date(b, yyyyMMdd)];
```

### Why are the changes needed?
Improve the error message

### Does this PR introduce _any_ user-facing change?
Yes, the error message will change

### How was this patch tested?
newly added test case

Closes #32831 from linhongliu-db/SPARK-35685-view-compatible.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 09:45:14 +00:00
allisonwang-db f281736fbd [SPARK-35618][SQL] Resolve star expressions in subqueries using outer query plans
### What changes were proposed in this pull request?
This PR supports resolving star expressions in subqueries using outer query plans.

### Why are the changes needed?
Currently, Spark can only resolve star expressions using the inner query plan when resolving subqueries. Instead, it should also be able to resolve star expressions using the outer query plans.

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

### How was this patch tested?
Unit tests

Closes #32787 from allisonwang-db/spark-35618-resolve-star-in-subquery.

Lead-authored-by: allisonwang-db <allison.wang@databricks.com>
Co-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 09:22:55 +00:00
Gengliang Wang f2492772ba [SPARK-35963][SQL] Rename TimestampWithoutTZType to TimestampNTZType
### What changes were proposed in this pull request?

Rename TimestampWithoutTZType to TimestampNTZType

### Why are the changes needed?

The time name of `TimestampWithoutTZType` is verbose. Rename it as `TimestampNTZType` so that
1. it is easier to read and type.
2. As we have the function to_timestamp_ntz, this makes the names consistent.
3. We will introduce a new SQL configuration `spark.sql.timestampType` for the default timestamp type. The configuration values can be "TIMESTMAP_NTZ" or "TIMESTMAP_LTZ" for simplicity.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Run `git grep -i WithoutTZ` and there is no result.
And Ci tests.

Closes #33167 from gengliangwang/rename.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 08:49:15 +00:00
Dongjoon Hyun 912d2b9834 [SPARK-35962][DOCS] Deprecate old Java 8 versions prior to 8u201
### What changes were proposed in this pull request?

This PR aims to deprecate old Java 8 versions prior to 8u201.

### Why are the changes needed?

This is a preparation of using G1GC during the migration among Java LTS versions (8/11/17).

8u162 has the following fix.
- JDK-8205376: JVM Crash during G1 GC

8u201 has the following fix.
- JDK-8208873: C1: G1 barriers don't preserve FP registers

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

No, Today's Java8 is usually 1.8.0_292 and this is just a deprecation in documentation.

### How was this patch tested?

N/A

Closes #33166 from dongjoon-hyun/SPARK-35962.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-01 15:45:32 +09:00
Dongjoon Hyun 74c4641e78 Revert "fix Spark version"
This reverts commit 6a2f4348ae.
2021-06-30 23:36:41 -07:00
Dongjoon Hyun 6a2f4348ae fix Spark version 2021-06-30 23:31:35 -07:00
ulysses-you ba0a479bda [SPARK-35961][SQL] Only use local shuffle reader when REBALANCE_PARTITIONS_BY_NONE without CustomShuffleReaderExec
### What changes were proposed in this pull request?

Remove dead code in `OptimizeLocalShuffleReader`.

### Why are the changes needed?

After [SPARK-35725](https://issues.apache.org/jira/browse/SPARK-35725), we might expand partition if that partition is skewed. So the partition number check `bytesByPartitionId.length == partitionSpecs.size` would be wrong if some partitions are coalesced and some partitions are splitted into smaller.
Note that, it's unlikely happened in real world since it used RoundRobin.

Otherhand, after [SPARK-34899](https://issues.apache.org/jira/browse/SPARK-34899), we use origin plan if can not coalesce partitions. So the assuming of that shuffle stage has `CustomShuffleReaderExec` with no effect is always false in `REBALANCE_PARTITIONS_BY_NONE` shuffle origin. That said, if no rule was efficient, there would be no `CustomShuffleReaderExec`.

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

No

### How was this patch tested?

Pass CI

Closes #33165 from ulysses-you/SPARK-35961.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 05:43:11 +00:00
Holden Karau 34286ae5bf [SPARK-35960][BUILD][TEST] Bump the scalatest version to 3.2.9
### What changes were proposed in this pull request?

Bump the scalatest version to 3.2.9

### Why are the changes needed?

With the scalatestplus change to 3.2.9.0, recent sbt fails to handle the mismatch between scalatest and scalatestplus and resolve resulting in test:compile errors of not being able to find the org.scalatest package.

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

No

### How was this patch tested?

sbt tags/test:compile failed before and passes with this change.

Closes #33163 from holdenk/SPARK-35960-test-compile-sbt-issue.

Authored-by: Holden Karau <hkarau@netflix.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 21:39:12 -07:00
Kevin Su dc85b0b51a [SPARK-35950][WEBUI] Failed to toggle Exec Loss Reason in the executors page
### What changes were proposed in this pull request?

Update the executor's page, so it can successfully hide the "Exec Loss Reason" column.

### Why are the changes needed?

When unselected the checkbox "Exec Loss Reason" on the executor page,
the "Active tasks" column disappears instead of the "Exec Loss Reason" column.

Before:
![Screenshot from 2021-06-30 15-55-05](https://user-images.githubusercontent.com/37936015/123930908-bd6f4180-d9c2-11eb-9aba-bbfe0a237776.png)
After:
![Screenshot from 2021-06-30 22-21-38](https://user-images.githubusercontent.com/37936015/123977632-bf042e00-d9f1-11eb-910e-93d615d2db47.png)

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

Yes, The Web UI is updated.

### How was this patch tested?

Pass the CIs.

Closes #33155 from pingsutw/SPARK-35950.

Lead-authored-by: Kevin Su <pingsutw@gmail.com>
Co-authored-by: Kevin Su <pingsutw@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-01 12:32:54 +08:00
yi.wu 868a594706 [SPARK-35714][FOLLOW-UP][CORE] Use a shared stopping flag for WorkerWatcher to avoid the duplicate System.exit
### What changes were proposed in this pull request?

This PR proposes to let `WorkerWatcher` reuse the `stopping` flag in `CoarseGrainedExecutorBackend` to avoid the duplicate call of `System.exit`.

### Why are the changes needed?

As a followup of https://github.com/apache/spark/pull/32868, this PR tries to give a more robust fix.

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

No.

### How was this patch tested?

Pass existing tests.

Closes #33028 from Ngone51/spark-35714-followup.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-07-01 11:40:00 +08:00
gengjiaan 5d74ace648 [SPARK-35065][SQL] Group exception messages in spark/sql (core)
### What changes were proposed in this pull request?
This PR group all exception messages in `sql/core/src/main/scala/org/apache/spark/sql`.

### 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 #32958 from beliefer/SPARK-35065.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 02:38:06 +00:00
Wenchen Fan cd6a463811 [SPARK-35888][SQL][FOLLOWUP] Return partition specs for all the shuffles
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33079, to fix a bug in corner cases: `ShufflePartitionsUtil.coalescePartitions` should either return the shuffle spec for all the shuffles, or none.

If the input RDD has no partition, the `mapOutputStatistics` is None, and we should still return shuffle specs with size 0.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

a new test

Closes #33158 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 01:43:11 +00:00
Takuya UESHIN a98c8ae57d [SPARK-35944][PYTHON] Introduce Name and Label type aliases
### What changes were proposed in this pull request?

Introduce `Name` and `Label` type aliases to distinguish what is expected instead of `Any` or `Union[Any, Tuple]`.

- `Label`: `Tuple[Any, ...]`
  Internal expression for name-like metadata, like `index_names`, `column_labels`, and `column_label_names` in `InternalFrame`, and similar internal structures.
- `Name`: `Union[Any, Label]`
  External expression for user-facing names, which can be scalar values or tuples.

### Why are the changes needed?

Currently `Any` or `Union[Any, Tuple]` is used for name-like types, but type aliases should be used to distinguish what is expected clearly.

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

No.

### How was this patch tested?

Existing tests.

Closes #33159 from ueshin/issues/SPARK-35944/name_and_label.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-01 09:40:07 +09:00
Xinrong Meng 5ad12611ec [SPARK-35938][PYTHON] Add deprecation warning for Python 3.6
### What changes were proposed in this pull request?

Add deprecation warning for Python 3.6.

### Why are the changes needed?

According to https://endoflife.date/python, Python 3.6 will be EOL on 23 Dec, 2021.
We should prepare for the deprecation of Python 3.6 support in Spark in advance.

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

N/A.

### How was this patch tested?

Manual tests.

Closes #33139 from xinrong-databricks/deprecate3.6_warn.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-01 09:32:25 +09:00
Xinrong Meng 9e39415f3a [SPARK-35939][DOCS][PYTHON] Deprecate Python 3.6 in Spark documentation
### What changes were proposed in this pull request?

Deprecate Python 3.6 in Spark documentation

### Why are the changes needed?

According to https://endoflife.date/python, Python 3.6 will be EOL on 23 Dec, 2021.
We should prepare for the deprecation of Python 3.6 support in Spark in advance.

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

N/A.

### How was this patch tested?

Manual tests.

Closes #33141 from xinrong-databricks/deprecate3.6_doc.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-01 09:31:34 +09:00
Chao Sun a5c886619d [SPARK-34859][SQL] Handle column index when using vectorized Parquet reader
### What changes were proposed in this pull request?

Make the current vectorized Parquet reader to work with column index introduced in Parquet 1.11. In particular, this PR makes the following changes:
1. in `ParquetReadState`, track row ranges returned via `PageReadStore.getRowIndexes` as well as the first row index for each page via `DataPage.getFirstRowIndex`.
1. introduced a new API `ParquetVectorUpdater.skipValues` which skips a batch of values from a Parquet value reader. As part of the process also renamed existing `updateBatch` to `readValues`, and `update` to `readValue` to keep the method names consistent.
1. in correspondence as above, also introduced new API `VectorizedValuesReader.skipXXX` for different data types, as well as the implementations. These are useful when the reader knows that the given batch of values can be skipped, for instance, due to the batch is not covered in the row ranges generated by column index filtering.
2. changed `VectorizedRleValuesReader` to handle column index filtering. This is done by comparing the range that is going to be read next within the current RLE/PACKED block (let's call this block range), against the current row range. There are three cases:
    * if the block range is before the current row range, skip all the values in the block range
    * if the block range is after the current row range, advance the row range and repeat the steps
    * if the block range overlaps with the current row range, only read the values within the overlapping area and skip the rest.

### Why are the changes needed?

[Parquet Column Index](https://github.com/apache/parquet-format/blob/master/PageIndex.md) is a new feature in Parquet 1.11 which allows very efficient filtering on page level (some benchmark numbers can be found [here](https://blog.cloudera.com/speeding-up-select-queries-with-parquet-page-indexes/)), especially when data is sorted. The feature is largely implemented in parquet-mr (via classes such as `ColumnIndex` and `ColumnIndexFilter`). In Spark, the non-vectorized Parquet reader can automatically benefit from the feature after upgrading to Parquet 1.11.x, without any code change. However, the same is not true for vectorized Parquet reader since Spark chose to implement its own logic such as reading Parquet pages, handling definition levels, reading values into columnar batches, etc.

Previously, [SPARK-26345](https://issues.apache.org/jira/browse/SPARK-26345) / (#31393) updated Spark to only scan pages filtered by column index from parquet-mr side. This is done by calling `ParquetFileReader.readNextFilteredRowGroup` and `ParquetFileReader.getFilteredRecordCount` API. The implementation, however, only work for a few limited cases: in the scenario where there are multiple columns and their type width are different (e.g., `int` and `bigint`), it could return incorrect result. For this issue, please see SPARK-34859 for a detailed description.

In order to fix the above, Spark needs to leverage the API `PageReadStore.getRowIndexes` and `DataPage.getFirstRowIndex`. The former returns the indexes of all rows (note the difference between rows and values: for flat schema there is no difference between the two, but for nested schema they're different) after filtering within a Parquet row group. The latter returns the first row index within a single data page. With the combination of the two, one is able to know which rows/values should be filtered while scanning a Parquet page.

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

Yes. Now the vectorized Parquet reader should work correctly with column index.

### How was this patch tested?

Borrowed tests from #31998 and added a few more tests.

Closes #32753 from sunchao/SPARK-34859.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Li Xian <lxian2shell@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 14:21:18 -07:00
ulysses-you d46c1e38ec [SPARK-35725][SQL] Support optimize skewed partitions in RebalancePartitions
### What changes were proposed in this pull request?

* Add a new rule `ExpandShufflePartitions` in AQE `queryStageOptimizerRules`
* Add a new config `spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled` to decide if should enable the new rule

The new rule `OptimizeSkewInRebalancePartitions` only handle two shuffle origin `REBALANCE_PARTITIONS_BY_NONE` and `REBALANCE_PARTITIONS_BY_COL` for data skew issue. And re-use the exists config `ADVISORY_PARTITION_SIZE_IN_BYTES` to decide what partition size should be.

### Why are the changes needed?

Currently, we don't support expand partition dynamically in AQE which is not friendly for some data skew job.

Let's say if we have a simple query:
```
SELECT /*+ REBALANCE(col) */ * FROM table
```

The column of `col` is skewed, then some shuffle partitions would handle too much data than others.

If we haven't inroduced extra shuffle, we can optimize this case by expanding partitions in AQE.

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

Yes, a new config

### How was this patch tested?

Add test

Closes #32883 from ulysses-you/expand-partition.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-30 18:04:50 +00:00
shane knapp 2c94fbc71e initial commit for skeleton ansible for jenkins worker config
### What changes were proposed in this pull request?
this is the skeleton of the ansible used to configure jenkins workers in the riselab/apache spark build system

### Why are the changes needed?
they are not needed, but will help the community understand how to build systems to test multiple versions of spark, as well as propose changes that i can integrate in to the "production" riselab repo.  since we're sunsetting jenkins by EOY 2021, this will potentially be useful for migrating the build system.

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

### How was this patch tested?
ansible-lint and much wailing and gnashing of teeth.

Closes #32178 from shaneknapp/initial-ansible-commit.

Lead-authored-by: shane knapp <incomplete@gmail.com>
Co-authored-by: shane <incomplete@gmail.com>
Signed-off-by: shane knapp <incomplete@gmail.com>
2021-06-30 10:05:27 -07:00
Gengliang Wang 733e85f1f4 [SPARK-35953][SQL] Support extracting date fields from timestamp without time zone
### What changes were proposed in this pull request?

Support extracting date fields from timestamp without time zone, which includes:
- year
- month
- day
- year of week
- week
- day of week
- quarter
- day of month
- day of year

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

No, the timestamp without time zone type is not released yet.

### How was this patch tested?

Unit tests

Closes #33156 from gengliangwang/dateField.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-01 00:44:48 +08:00
Angerszhuuuu 2febd5c3f0 [SPARK-35735][SQL] Take into account day-time interval fields in cast
### What changes were proposed in this pull request?
Support take into account day-time interval field in cast.

### Why are the changes needed?
To conform to the SQL standard.

### Does this PR introduce _any_ user-facing change?
An user can use `cast(str, DayTimeInterval(DAY, HOUR))`, for instance.

### How was this patch tested?
Added UT.

Closes #32943 from AngersZhuuuu/SPARK-35735.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-30 16:05:04 +03:00
Gengliang Wang e88aa49287 [SPARK-35932][SQL] Support extracting hour/minute/second from timestamp without time zone
### What changes were proposed in this pull request?

Support extracting hour/minute/second fields from timestamp without time zone values. In details, the following syntaxes are supported:

- extract [hour | minute | second] from timestampWithoutTZ
- date_part('[hour | minute | second]', timestampWithoutTZ)
- hour(timestampWithoutTZ)
- minute(timestampWithoutTZ)
- second(timestampWithoutTZ)

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

No, the timestamp without time zone type is not release yet.

### How was this patch tested?

Unit test

Closes #33136 from gengliangwang/field.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-30 19:36:47 +08:00
Gengliang Wang c6afd6ed52 [SPARK-35951][DOCS] Add since versions for Avro options in Documentation
### What changes were proposed in this pull request?

There are two new Avro options `datetimeRebaseMode` and `positionalFieldMatching` after Spark 3.2.
We should document the since version so that users can know whether the option works in their Spark version.

### Why are the changes needed?

Better documentation.

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

No
### How was this patch tested?

Manual preview on local setup.
<img width="828" alt="Screen Shot 2021-06-30 at 5 05 54 PM" src="https://user-images.githubusercontent.com/1097932/123934000-ba833b00-d947-11eb-9ca5-ce8ff8add74b.png">

<img width="711" alt="Screen Shot 2021-06-30 at 5 06 34 PM" src="https://user-images.githubusercontent.com/1097932/123934126-d4bd1900-d947-11eb-8d80-69df8f3d9900.png">

Closes #33153 from gengliangwang/version.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-30 17:24:48 +08:00
Karen Feng e3bd817d65 [SPARK-34920][CORE][SQL] Add error classes with SQLSTATE
### What changes were proposed in this pull request?

Unifies exceptions thrown from Spark under a single base trait `SparkError`, which unifies:
- Error classes
- Parametrized error messages
- SQLSTATE, as discussed in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Add-error-IDs-td31126.html.

### Why are the changes needed?

- Adding error classes creates a consistent label for exceptions, even as error messages change
- Creating a single, centralized source-of-truth for parametrized error messages improves auditing for error message quality
- Adding SQLSTATE helps ODBC/JDBC users receive standardized error codes

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

Yes, changes ODBC experience by:
- Adding error classes to error messages
- Adding SQLSTATE to TStatus

### How was this patch tested?

Unit tests, as well as local tests with PyODBC.

Closes #32850 from karenfeng/SPARK-34920.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-30 09:22:02 +00:00
Erik Krogen 4dd41b9678 [SPARK-34365][AVRO] Add support for positional Catalyst-to-Avro schema matching
### What changes were proposed in this pull request?
Provide the (configurable) ability to perform Avro-to-Catalyst schema field matching using the position of the fields instead of their names. A new `option` is added for the Avro datasource, `positionalFieldMatching`, which instructs `AvroSerializer`/`AvroDeserializer` to perform positional field matching instead of matching by name.

### Why are the changes needed?
This by-name matching is somewhat recent; prior to PR #24635, at least on the write path, schemas were matched by positionally ("structural" comparison). While by-name is better behavior as a default, it will be better to make this configurable by a user. Even at the time that PR #24635 was handled, there was [interest in making this behavior configurable](https://github.com/apache/spark/pull/24635#issuecomment-494205251), but it appears it went unaddressed.

There is precedence for configurability of this behavior as seen in PR #29737, which added this support for ORC. Besides this precedence, the behavior of Hive is to perform matching positionally ([ref](https://cwiki.apache.org/confluence/display/Hive/AvroSerDe#AvroSerDe-WritingtablestoAvrofiles)), so this is behavior that Hadoop/Hive ecosystem users are familiar with.

### Does this PR introduce _any_ user-facing change?
Yes, a new option is provided for the Avro datasource, `positionalFieldMatching`, which provides compatibility with Hive and pre-3.0.0 Spark behavior.

### How was this patch tested?
New unit tests are added within `AvroSuite`, `AvroSchemaHelperSuite`, and `AvroSerdeSuite`; and most of the existing tests within `AvroSerdeSuite` are adapted to perform the same test using by-name and positional matching to ensure feature parity.

Closes #31490 from xkrogen/xkrogen-SPARK-34365-avro-positional-field-matching.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-30 16:20:45 +08:00
Cheng Su 6bbfb45ffe [SPARK-33298][CORE][FOLLOWUP] Add Unstable annotation to FileCommitProtocol
### What changes were proposed in this pull request?

This is the followup from https://github.com/apache/spark/pull/33012#discussion_r659440833, where we want to add `Unstable` to `FileCommitProtocol`, to give people a better idea of API.

### Why are the changes needed?

Make it easier for people to follow and understand code. Clean up code.

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

No.

### How was this patch tested?

Existing unit tests, as no real logic change.

Closes #33148 from c21/bucket-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-30 16:25:20 +09:00
Dongjoon Hyun b218cc90cf [SPARK-35948][INFRA] Simplify release scripts by removing Spark 2.4/Java7 parts
### What changes were proposed in this pull request?

This PR aims to clean up Spark 2.4 and Java7 code path from the release scripts.

### Why are the changes needed?

To simplify the logic.

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

No.

### How was this patch tested?

N/A

Closes #33150 from dongjoon-hyun/SPARK-35948.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-30 16:24:03 +09:00