Commit graph

29977 commits

Author SHA1 Message Date
Angerszhuuuu 8ed5808f64 [SPARK-34488][CORE] Support task Metrics Distributions and executor Metrics Distributions in the REST API call for a specified stage
### What changes were proposed in this pull request?
For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json

Similarly, it is useful to show the executor metrics in percentile distribution for a specific stage. This information can show whether or not there is a skewed load on some executors.  We list an example in executorMetricsDistributions.json

We define `withSummaries` and `quantiles` query parameter in the REST API for a specific stage as:

applications/<application_id>/<application_attempt/stages/<stage_id>/<stage_attempt>?withSummaries=[true|false]& quantiles=0.05,0.25,0.5,0.75,0.95

1. withSummaries: default is false, define whether to show current stage's taskMetricsDistribution and executorMetricsDistribution
2. quantiles: default is `0.0,0.25,0.5,0.75,1.0` only effect when `withSummaries=true`, it define the quantiles we use when calculating metrics distributions.

When withSummaries=true, both task metrics in percentile distribution and executor metrics in percentile distribution are included in the REST API output.  The default value of withSummaries is false, i.e. no metrics percentile distribution will be included in the REST API output.

 

### Why are the changes needed?
For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json

### Does this PR introduce _any_ user-facing change?
User can  use  below restful API to get task metrics distribution and executor metrics distribution for indivial stage
```
applications/<application_id>/<application_attempt/stages/<stage_id>/<stage_attempt>?withSummaries=[true|false]
```

### How was this patch tested?
Added UT

Closes #31611 from AngersZhuuuu/SPARK-34488.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-24 08:50:45 -05:00
Liang-Chi Hsieh 95c61df0fa [SPARK-34295][CORE] Exclude filesystems from token renewal at YARN
### What changes were proposed in this pull request?

This patch adds a config `spark.yarn.kerberos.renewal.excludeHadoopFileSystems` which lists the filesystems to be excluded from delegation token renewal at YARN.

### Why are the changes needed?

MapReduce jobs can instruct YARN to skip renewal of tokens obtained from certain hosts by specifying the hosts with configuration mapreduce.job.hdfs-servers.token-renewal.exclude=<host1>,<host2>,..,<hostN>.

But seems Spark lacks of similar option. So the job submission fails if YARN fails to renew DelegationToken for any of the remote HDFS cluster. The failure in DT renewal can happen due to many reason like Remote HDFS does not trust Kerberos identity of YARN etc. We have a customer facing such issue.

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

No, if the config is not set. Yes, as users can use this config to instruct YARN not to renew delegation token from certain filesystems.

### How was this patch tested?

It is hard to do unit test for this. We did verify it work from the customer using this fix in the production environment.

Closes #31761 from viirya/SPARK-34295.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-24 01:11:53 -07:00
Shardul Mahadik 2298cebcf8 [SPARK-34477][CORE] Register KryoSerializers for Avro GenericData classes
### What changes were proposed in this pull request?
1) Modify `GenericAvroSerializer` to support serialization of any `GenericContainer`
2) Register `KryoSerializer`s for `GenericData.{Array, EnumSymbol, Fixed}` using the modified `GenericAvroSerializer`

### Why are the changes needed?
Without this change, Kryo throws NPEs when trying to serialize `GenericData.{Array, EnumSymbol, Fixed}`. More details in SPARK-34477 Jira

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

No

### How was this patch tested?

Added unit tests for testing roundtrip serialization and deserialization of `GenericData.{Array, EnumSymbol, Fixed}` using `GenericAvroSerializer` directly and also indirectly through `KryoSerializer`

Closes #31597 from shardulm94/avro-array-serializer.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-24 15:31:34 +08:00
yangjie01 712a62ca82 [SPARK-34832][SQL][TEST] Set EXECUTOR_ALLOW_SPARK_CONTEXT to true to ensure ExternalAppendOnlyUnsafeRowArrayBenchmark run successfully
### What changes were proposed in this pull request?
SPARK-32160 add a config(`EXECUTOR_ALLOW_SPARK_CONTEXT`) to switch allow/disallow to create `SparkContext` in executors and the default value of the config is `false`

`ExternalAppendOnlyUnsafeRowArrayBenchmark` will run fail when `EXECUTOR_ALLOW_SPARK_CONTEXT` use the default value because the `ExternalAppendOnlyUnsafeRowArrayBenchmark#withFakeTaskContext` method try to create a `SparkContext` manually in Executor Side.

So the main change of this pr is  set `EXECUTOR_ALLOW_SPARK_CONTEXT` to `true` to ensure `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

### Why are the changes needed?
Bug fix.

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

### How was this patch tested?
Manual test:
```
bin/spark-submit --class org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark --jars spark-core_2.12-3.2.0-SNAPSHOT-tests.jar spark-sql_2.12-3.2.0-SNAPSHOT-tests.jar
```

**Before**
```
Exception in thread "main" java.lang.IllegalStateException: SparkContext should only be created and accessed on the driver.
	at org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$assertOnDriver(SparkContext.scala:2679)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:89)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:137)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.withFakeTaskContext(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:52)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.testAgainstRawArrayBuffer(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:119)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.$anonfun$runBenchmarkSuite$1(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:189)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.benchmark.BenchmarkBase.runBenchmark(BenchmarkBase.scala:40)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.runBenchmarkSuite(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:186)
	at org.apache.spark.benchmark.BenchmarkBase.main(BenchmarkBase.scala:58)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark.main(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

**After**

`ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

Closes #31939 from LuciferYang/SPARK-34832.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-24 14:59:31 +09:00
Kousuke Saruta f7e9b6efc7 [SPARK-34763][SQL] col(), $"<name>" and df("name") should handle quoted column names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `col()`, `$"<name>"` and `df("name")` don't handle quoted column names  like ``` `a``b.c` ```properly.

For example, if we have a following DataFrame.
```
val df1 = spark.sql("SELECT 'col1' AS `a``b.c`")
```

For the DataFrame, this query is successfully executed.
```
scala> df1.selectExpr("`a``b.c`").show
+-----+
|a`b.c|
+-----+
| col1|
+-----+
```

But the following query will fail because ``` df1("`a``b.c`") ``` throws an exception.
```
scala> df1.select(df1("`a``b.c`")).show
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `a``b.c`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:152)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:162)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221)
  at org.apache.spark.sql.Dataset.col(Dataset.scala:1274)
  at org.apache.spark.sql.Dataset.apply(Dataset.scala:1241)
  ... 49 elided
```
### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

New tests.

Closes #31854 from sarutak/fix-parseAttributeName.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 13:34:10 +08:00
hongdongdong 985c653b20 [SPARK-33720][K8S] Support submit to k8s only with token
### What changes were proposed in this pull request?

Support submit to k8s only with token.

### Why are the changes needed?

Now, sumbit to k8s always need oauth files.

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

### How was this patch tested?

Before, submit job out of k8s cluster without correct ca.crt, we may get this exception:
```
Caused by: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
        at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:439)
        at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:306)
        at sun.security.validator.Validator.validate(Validator.java:271)
        at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:312)
```
When set spark.kubernetes.trust.certificates = true, we can submit only with correct token, no need to config ca.crt in local env.
Submit as:
```
 bin/spark-submit \
     --master $master \
     --name pi \
     --deploy-mode cluster \
     --conf spark.kubernetes.container.image=$image \
     --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \
     --conf spark.kubernetes.authenticate.submission.oauthToken=$clusterToken \
     --conf spark.kubernetes.trust.certificates=true \
     local:///opt/spark/examples/src/main/python/pi.py 200
```

Closes #30684 from hddong/trust-certs.

Authored-by: hongdongdong <hongdongdong@cmss.chinamobile.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-23 22:07:27 -07:00
Takeshi Yamamuro 0494dc90af [SPARK-34842][SQL][TESTS] Corrects the type of date_dim.d_quarter_name in the TPCDS schema
### What changes were proposed in this pull request?

SPARK-34842 (#31012) has a typo in the type of `date_dim.d_quarter_name` in the TPCDS schema (`TPCDSBase`). This PR replace `CHAR(1)` with `CHAR(6)`. This fix comes from p28 in [the TPCDS official doc](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

N/A

Closes #31943 from maropu/SPARK-34083-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-23 10:22:13 -07:00
Max Gekk 760556a42f [SPARK-34824][SQL] Support multiply an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31929 from MaxGekk/interval-mul-div.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-23 19:40:15 +03:00
robert4os 06d40696dc [MINOR][DOCS] Update sql-ref-syntax-dml-insert-into.md
### What changes were proposed in this pull request?

the given example uses a non-standard syntax for CREATE TABLE, by defining the partitioning column with the other columns, instead of in PARTITION BY.

This works is this case, because the partitioning column happens to be the last column defined, but it will break if instead 'name' would be used for partitioning.

I suggest therefore to change the example to use a standard syntax, like in
https://spark.apache.org/docs/3.1.1/sql-ref-syntax-ddl-create-table-hiveformat.html

### Why are the changes needed?

To show the better documentation.

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

Yes, this fixes the user-facing docs.

### How was this patch tested?

CI should test it out.

Closes #31900 from robert4os/patch-1.

Authored-by: robert4os <robert4os@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-24 00:13:06 +09:00
Wenchen Fan 3b70829b5b [SPARK-34719][SQL] Correctly resolve the view query with duplicated column names
forward-port https://github.com/apache/spark/pull/31811 to master

### What changes were proposed in this pull request?

For permanent views (and the new SQL temp view in Spark 3.1), we store the view SQL text and re-parse/analyze the view SQL text when reading the view. In the case of `SELECT * FROM ...`, we want to avoid view schema change (e.g. the referenced table changes its schema) and will record the view query output column names when creating the view, so that when reading the view we can add a `SELECT recorded_column_names FROM ...` to retain the original view query schema.

In Spark 3.1 and before, the final SELECT is added after the analysis phase: https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala#L67

If the view query has duplicated output column names, we always pick the first column when reading a view. A simple repro:
```
scala> sql("create view c(x, y) as select 1 a, 2 a")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("select * from c").show
+---+---+
|  x|  y|
+---+---+
|  1|  1|
+---+---+
```

In the master branch, we will fail at the view reading time due to b891862fb6 , which adds the final SELECT during analysis, so that the query fails with `Reference 'a' is ambiguous`

This PR proposes to resolve the view query output column names from the matching attributes by ordinal.

For example,  `create view c(x, y) as select 1 a, 2 a`, the view query output column names are `[a, a]`. When we reading the view, there are 2 matching attributes (e.g.`[a#1, a#2]`) and we can simply match them by ordinal.

A negative example is
```
create table t(a int)
create view v as select *, 1 as col from t
replace table t(a int, col int)
```
When reading the view, the view query output column names are `[a, col]`, and there are two matching attributes of `col`, and we should fail the query. See the tests for details.

### Why are the changes needed?

bug fix

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

yes

### How was this patch tested?

new test

Closes #31930 from cloud-fan/view2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 14:34:51 +00:00
Liang-Chi Hsieh 115ed89a3c [SPARK-34366][SQL] Add interface for DS v2 metrics
### What changes were proposed in this pull request?

This patch proposes to add a few public API change to DS v2, to make DS v2 scan can report metrics to Spark.

Two public interfaces are added.

* `CustomMetric`: metric interface at the driver side. It basically defines how Spark aggregates task metrics with the same metric name.
* `CustomTaskMetric`: task metric reported at executors. It includes a name and long value. Spark will collect these metric values and update internal metrics.

There are two public methods added to existing public interfaces. They are optional to DS v2 implementations.

* `PartitionReader.currentMetricsValues()`: returns an array of CustomTaskMetric. Here is where the actual metrics values are collected. Empty array by default.
* `Scan.supportedCustomMetrics()`: returns an array of supported custom metrics `CustomMetric`. Empty array by default.

### Why are the changes needed?

In order to report custom metrics, we need some public API change in DS v2 to make it possible.

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

No

### How was this patch tested?

This only adds interfaces. In follow-up PRs where adding implementation there will be tests added. See #31451 and #31398 for some details and manual test there.

Closes #31476 from viirya/SPARK-34366.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 13:22:37 +00:00
Peter Toth 93a5d34f84 [SPARK-33482][SPARK-34756][SQL] Fix FileScan equality check
### What changes were proposed in this pull request?

This bug was introduced by SPARK-30428 at Apache Spark 3.0.0.
This PR fixes `FileScan.equals()`.

### Why are the changes needed?
- Without this fix `FileScan.equals` doesn't take `fileIndex` and `readSchema` into account.
- Partition filters and data filters added to `FileScan` (in #27112 and #27157) caused that canonicalized form of some `BatchScanExec` nodes don't match and this prevents some reuse possibilities.

### Does this PR introduce _any_ user-facing change?
Yes, before this fix incorrect reuse of `FileScan` and so `BatchScanExec` could have happed causing correctness issues.

### How was this patch tested?
Added new UTs.

Closes #31848 from peter-toth/SPARK-34756-fix-filescan-equality-check.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 17:01:16 +08:00
yi.wu e00afd31a7 [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself
### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

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

No.

### How was this patch tested?

Pass existing tests.

Closes #31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:38:43 +00:00
Lena d32bb4e5ee [MINOR][DOCS] Updating the link for Azure Data Lake Gen 2 in docs
### What changes were proposed in this pull request?

Current link for `Azure Blob Storage and Azure Datalake Gen 2` leads to AWS information. Replacing the link to point to the right page.

### Why are the changes needed?

For users to access to the correct link.

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

Yes, it fixes the link correctly.

### How was this patch tested?

N/A

Closes #31938 from lenadroid/patch-1.

Authored-by: Lena <alehall@microsoft.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-23 10:13:32 +03:00
linzebing e768eaa908 [SPARK-34707][SQL] Code-gen broadcast nested loop join (left outer/right outer)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left outer (build right) and right outer (build left). Reference: `BroadcastNestedLoopJoinExec.codegenInner()` and `BroadcastNestedLoopJoinExec.outerJoin()`

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:
```scala
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left outer broadcast nested loop join", N) {
   val df = spark.range(N).selectExpr(s"id as k1").join(
     dim, col("k1") + 1 <= col("k2"), "left_outer")
   assert(df.queryExecution.sparkPlan.find(
     _.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined)
   df.noop()
}
```
Seeing 2x run time improvement:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left outer broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------------
left outer broadcast nested loop join wholestage off           3024           3698         953          6.9         144.2       1.0X
left outer broadcast nested loop join wholestage on            1512           1659         172         13.9          72.1       2.0X
```

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

No

### How was this patch tested?

Changed existing unit tests in `OuterJoinSuite` to cover codegen use cases.
Added unit test in WholeStageCodegenSuite.scala to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:
```scala
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_outer").explain("codegen")
```
Example generated code (`bnlj_doConsume_0` method):
```java
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, LeftOuter, ((k1#2L + 1) <= k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=16)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=16)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_foundMatch_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */       boolean bnlj_shouldOutputRow_0 = false;
/* 041 */
/* 042 */       boolean bnlj_isNull_2 = true;
/* 043 */       long bnlj_value_2 = -1L;
/* 044 */       if (bnlj_buildRow_0 != null) {
/* 045 */         long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 046 */         bnlj_isNull_2 = false;
/* 047 */         bnlj_value_2 = bnlj_value_1;
/* 048 */       }
/* 049 */
/* 050 */       long bnlj_value_4 = -1L;
/* 051 */
/* 052 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 053 */
/* 054 */       boolean bnlj_value_3 = false;
/* 055 */       bnlj_value_3 = bnlj_value_4 <= bnlj_value_2;
/* 056 */       if (!(false || !bnlj_value_3))
/* 057 */       {
/* 058 */         bnlj_shouldOutputRow_0 = true;
/* 059 */         bnlj_foundMatch_0 = true;
/* 060 */       }
/* 061 */       if (bnlj_arrayIndex_0 == bnlj_buildRowArray_0.length - 1 && !bnlj_foundMatch_0) {
/* 062 */         bnlj_buildRow_0 = null;
/* 063 */         bnlj_shouldOutputRow_0 = true;
/* 064 */       }
/* 065 */       if (bnlj_shouldOutputRow_0) {
/* 066 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 067 */
/* 068 */         boolean bnlj_isNull_9 = true;
/* 069 */         long bnlj_value_9 = -1L;
/* 070 */         if (bnlj_buildRow_0 != null) {
/* 071 */           long bnlj_value_8 = bnlj_buildRow_0.getLong(0);
/* 072 */           bnlj_isNull_9 = false;
/* 073 */           bnlj_value_9 = bnlj_value_8;
/* 074 */         }
/* 075 */         range_mutableStateArray_0[3].reset();
/* 076 */
/* 077 */         range_mutableStateArray_0[3].zeroOutNullBytes();
/* 078 */
/* 079 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 080 */
/* 081 */         if (bnlj_isNull_9) {
/* 082 */           range_mutableStateArray_0[3].setNullAt(1);
/* 083 */         } else {
/* 084 */           range_mutableStateArray_0[3].write(1, bnlj_value_9);
/* 085 */         }
/* 086 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 087 */
/* 088 */       }
/* 089 */     }
/* 090 */
/* 091 */   }
/* 092 */
/* 093 */   private void initRange(int idx) {
/* 094 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 095 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(16L);
/* 096 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 097 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 098 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 099 */     long partitionEnd;
/* 100 */
/* 101 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 102 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 103 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 104 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 105 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 106 */     } else {
/* 107 */       range_nextIndex_0 = st.longValue();
/* 108 */     }
/* 109 */     range_batchEnd_0 = range_nextIndex_0;
/* 110 */
/* 111 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 112 */     .multiply(step).add(start);
/* 113 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 114 */       partitionEnd = Long.MAX_VALUE;
/* 115 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 116 */       partitionEnd = Long.MIN_VALUE;
/* 117 */     } else {
/* 118 */       partitionEnd = end.longValue();
/* 119 */     }
/* 120 */
/* 121 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 122 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 123 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 124 */     if (range_numElementsTodo_0 < 0) {
/* 125 */       range_numElementsTodo_0 = 0;
/* 126 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 127 */       range_numElementsTodo_0++;
/* 128 */     }
/* 129 */   }
/* 130 */
/* 131 */   protected void processNext() throws java.io.IOException {
/* 132 */     // initialize Range
/* 133 */     if (!range_initRange_0) {
/* 134 */       range_initRange_0 = true;
/* 135 */       initRange(partitionIndex);
/* 136 */     }
/* 137 */
/* 138 */     while (true) {
/* 139 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 140 */         long range_nextBatchTodo_0;
/* 141 */         if (range_numElementsTodo_0 > 1000L) {
/* 142 */           range_nextBatchTodo_0 = 1000L;
/* 143 */           range_numElementsTodo_0 -= 1000L;
/* 144 */         } else {
/* 145 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 146 */           range_numElementsTodo_0 = 0;
/* 147 */           if (range_nextBatchTodo_0 == 0) break;
/* 148 */         }
/* 149 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 150 */       }
/* 151 */
/* 152 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 153 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 154 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 155 */
/* 156 */         // common sub-expressions
/* 157 */
/* 158 */         bnlj_doConsume_0(range_value_0);
/* 159 */
/* 160 */         if (shouldStop()) {
/* 161 */           range_nextIndex_0 = range_value_0 + 1L;
/* 162 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 163 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 164 */           return;
/* 165 */         }
/* 166 */
/* 167 */       }
/* 168 */       range_nextIndex_0 = range_batchEnd_0;
/* 169 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 170 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 171 */       range_taskContext_0.killTaskIfInterrupted();
/* 172 */     }
/* 173 */   }
/* 174 */
/* 175 */ }
```

Closes #31931 from linzebing/code-left-right-outer.

Authored-by: linzebing <linzebing1995@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:11:57 +00:00
hezuojiao 39542bb81f [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled
### What changes were proposed in this pull request?

This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption.

### Why are the changes needed?
Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message:
```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message=
org.apache.spark.shuffle.FetchFailedException: Stream is corrupted
	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845)
	at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
	at java.io.DataInputStream.readInt(DataInputStream.java:387)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:494)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
	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:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.io.IOException: Stream is corrupted
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200)
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226)
	at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841)
	... 25 more

)
```

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

No

### How was this patch tested?

New tests.

Closes #31898 from hezuojiao/fetch_shuffle_in_batch.

Authored-by: hezuojiao <hezuojiao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 13:06:12 -07:00
Yikun Jiang 31da90762e [SPARK-34820][K8S][R] add apt-update before gnupg install
### What changes were proposed in this pull request?
We added the gnupg installation in https://github.com/apache/spark/pull/30130 , we should do apt update before gnupg isntallation, otherwise we will get a fetch error when package is updated.

See more in:
[1] http://apache-spark-developers-list.1001551.n3.nabble.com/K8s-Integration-test-is-unable-to-run-because-of-the-unavailable-libs-td30986.html

### Why are the changes needed?
add a apt-update cmd before gnupg installation to avoid invaild package cache list.

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

### How was this patch tested?
K8s Integration test passed

Closes #31923 from Yikun/SPARK-34820.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 10:13:31 -07:00
PengLei 85581f6dac [SPARK-33925][CORE][FOLLOW-UP] Remove the unused variables 'secMgr'
### What changes were proposed in this pull request?
Remove the unused variable 'secMgr' in SparkSubmit.scala and DriverWrapper.scala
In jira https://issues.apache.org/jira/browse/SPARK-33925, The last usage of SecurityManager in Utils.fetchFile was removed. We don't need the variable anymore

### Why are the changes needed?
For better readablity of codes

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

### How was this patch tested?
Manually complied. Github Actions and Jenkins build should test it out as well.

Closes #31928 from Peng-Lei/rm_secMgr.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-22 12:02:25 -05:00
tanel.kiis@gmail.com 51cf0cadea [SPARK-34812][SQL] RowNumberLike and RankLike should not be nullable
### What changes were proposed in this pull request?

Marked `RowNumberLike` and `RankLike` as not-nullable.

### Why are the changes needed?

`RowNumberLike` and `RankLike` SQL expressions never return null value. Marking them as non-nullable can have some performance benefits, because some optimizer rules apply only to non-nullable expressions

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

No

### How was this patch tested?

Did not find any existing tests on the nullability of aggregate functions.
Plan stability suite partially covers this.

Closes #31924 from tanelk/SPARK-34812_nullability.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 14:55:43 +00:00
John Ayad ddfc75ec64 [SPARK-34803][PYSPARK] Pass the raised ImportError if pandas or pyarrow fail to import
### What changes were proposed in this pull request?

Pass the raised `ImportError` on failing to import pandas/pyarrow. This will help the user identify whether pandas/pyarrow are indeed not in the environment or if they threw a different `ImportError`.

### Why are the changes needed?

This can already happen in Pandas for example where it could throw an `ImportError` on its initialisation path if `dateutil` doesn't satisfy a certain version requirement https://github.com/pandas-dev/pandas/blob/0.24.x/pandas/compat/__init__.py#L438

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

Yes, it will now show the root cause of the exception when pandas or arrow is missing during import.

### How was this patch tested?

Manually tested.

```python
from pyspark.sql.functions import pandas_udf
spark.range(1).select(pandas_udf(lambda x: x))
```

Before:

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/...//spark/python/pyspark/sql/pandas/functions.py", line 332, in pandas_udf
    require_minimum_pyarrow_version()
  File "/.../spark/python/pyspark/sql/pandas/utils.py", line 53, in require_minimum_pyarrow_version
    raise ImportError("PyArrow >= %s must be installed; however, "
ImportError: PyArrow >= 1.0.0 must be installed; however, it was not found.
```

After:

```
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/pandas/utils.py", line 49, in require_minimum_pyarrow_version
    import pyarrow
ModuleNotFoundError: No module named 'pyarrow'

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/pandas/functions.py", line 332, in pandas_udf
    require_minimum_pyarrow_version()
  File "/.../spark/python/pyspark/sql/pandas/utils.py", line 55, in require_minimum_pyarrow_version
    raise ImportError("PyArrow >= %s must be installed; however, "
ImportError: PyArrow >= 1.0.0 must be installed; however, it was not found.
```

Closes #31902 from johnhany97/jayad/spark-34803.

Lead-authored-by: John Ayad <johnhany97@gmail.com>
Co-authored-by: John H. Ayad <johnhany97@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-22 23:29:28 +09:00
Ismaël Mejía 8a552bfc76 [SPARK-34778][BUILD] Upgrade to Avro 1.10.2
### What changes were proposed in this pull request?
Update the  Avro version to 1.10.2

### Why are the changes needed?
To stay up to date with upstream and catch compatibility issues with zstd

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

### How was this patch tested?
Unit tests

Closes #31866 from iemejia/SPARK-27733-upgrade-avro-1.10.2.

Authored-by: Ismaël Mejía <iemejia@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-22 19:30:14 +08:00
woyumen4597 f44608a8c0 [SPARK-34800][SQL] Use fine-grained lock in SessionCatalog.tableExists
### What changes were proposed in this pull request?
Use fine-grained lock in SessionCatalog.tableExists, in order to lock currentDB variable rather than lock `tableExists` method which will block inner external catalog's behaviour.

### Why are the changes needed?
We have modified the underlying hive meta store which a different hive  database is placed in its own shard for performance. However, we found that the synchronized lock  limits the concurrency.

### How was this patch tested?
Existing tests.

Closes #31891 from woyumen4597/SPARK-34800.

Authored-by: woyumen4597 <woyumen4597@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 09:03:46 +00:00
Terry Kim 7953fcdb56 [SPARK-34700][SQL] SessionCatalog's temporary view related APIs should take/return more concrete types
### What changes were proposed in this pull request?

Now that all the temporary views are wrapped with `TemporaryViewRelation`(#31273, #31652, and #31825), this PR proposes to update `SessionCatalog`'s APIs for temporary views to take or return more concrete types.

APIs that will take `TemporaryViewRelation` instead of `LogicalPlan`:
```
createTempView, createGlobalTempView, alterTempViewDefinition
```

APIs that will return `TemporaryViewRelation` instead of `LogicalPlan`:
```
getRawTempView, getRawGlobalTempView
```

APIs that will return `View` instead of `LogicalPlan`:
```
getTempView, getGlobalTempView, lookupTempView
```

### Why are the changes needed?

Internal refactoring to work with more concrete types.

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

No, this is internal refactoring.

### How was this patch tested?

Updated existing tests affected by the refactoring.

Closes #31906 from imback82/use_temporary_view_relation.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:17:54 +00:00
yi.wu e4bb97526c [SPARK-34089][CORE] HybridRowQueue should respect the configured memory mode
### What changes were proposed in this pull request?

This PR fixes the `HybridRowQueue ` to respect the configured memory mode.

Besides, this PR also refactored the constructor of `MemoryConsumer` to accept the memory mode explicitly.

### Why are the changes needed?

`HybridRowQueue` supports both onHeap and offHeap manipulation. But it inherited the wrong `MemoryConsumer` constructor, which hard-coded the memory mode to `onHeap`.

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

No. (Maybe yes in some cases where users can't complete the job before could complete successfully after the fix because of `HybridRowQueue` is able to spill under offHeap mode now. )

### How was this patch tested?

Updated the existing test to make it test both offHeap and onHeap modes.

Closes #31152 from Ngone51/fix-MemoryConsumer-memorymode.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:12:08 +00:00
HyukjinKwon ec70467d4d [SPARK-34815][SQL] Update CSVBenchmark
### What changes were proposed in this pull request?

This PR updates CSVBenchmark especially we have a fix like https://github.com/apache/spark/pull/31858 that could potentially improve the performance.

### Why are the changes needed?

To have the updated benchmark results.

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

No.

### How was this patch tested?

Manually ran the benchmark

Closes #31917 from HyukjinKwon/SPARK-34815.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 10:49:53 +03:00
Jungtaek Lim (HeartSaVioR) 121883b1a5 [SPARK-34383][SS] Optimize WAL commit phase via reducing cost of filesystem operations
### What changes were proposed in this pull request?

This PR proposes to optimize WAL commit phase via following changes:

* cache offset log to avoid FS get operation per batch
* just directly delete instead of employing FS list operation on purge

### Why are the changes needed?

There're inefficiency on WAL commit phase which can be easily optimized via using a small driver memory.

1. To provide the offset metadata to source side (via `source.commit()`), we read offset metadata for previous batch from file system, which is probably written by this driver in previous batches. Caching it into driver memory would reduce the get operation.
2. Spark calls purge against offset log & commit log per batch, which calls list operation. If the previous batch succeeded to purge, the current batch just needs to check one batch which can be simply done via direct delete operation, instead of calling list operation.

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

No.

### How was this patch tested?

Manually tested with additional debug log. (Verified that cache is used, cache keeps the size as 2, only one delete call is used instead of list call)

Did some experiment with simple rate to console query. (NOTE: wasn't done with master branch - tested against Spark 2.4.x, but WAL commit phase hasn't been changed AFAIK during these versions)

AWS S3 + S3 guard:

> before the patch

<img width="1075" alt="aws-before" src="https://user-images.githubusercontent.com/1317309/107108721-6cc54380-687d-11eb-8f10-b906b9d58397.png">

> after the patch

<img width="1071" alt="aws-after" src="https://user-images.githubusercontent.com/1317309/107108724-7189f780-687d-11eb-88da-26912ac15c85.png">

Azure:

> before the patch

<img width="1074" alt="azure-before" src="https://user-images.githubusercontent.com/1317309/107108726-75b61500-687d-11eb-8c06-9048fa10ff9a.png">

> after the patch

<img width="1069" alt="azure-after" src="https://user-images.githubusercontent.com/1317309/107108729-79e23280-687d-11eb-8d97-e7f3aeec51be.png">

Closes #31495 from HeartSaVioR/SPARK-34383.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
2021-03-22 08:47:07 +01:00
Cheng Su f8838fe82b [SPARK-34708][SQL] Code-gen for left semi/anti broadcast nested loop join (build right side)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left semi / left anti BroadcastNestedLoopJoin (build side is right side). The execution code path for build left side cannot fit into whole stage code-gen framework, so only add the code-gen for build right side here.

Reference: the iterator (non-code-gen) code path is `BroadcastNestedLoopJoinExec.leftExistenceJoin()` with `BuildRight`.

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:

```
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left semi broadcast nested loop join", N) {
  park.range(N).selectExpr(s"id as k1").join(
    dim, col("k1") + 1 <= col("k2"), "left_semi")
}
```

Seeing 5x run time improvement:

```
Running benchmark: left semi broadcast nested loop join
  Running case: left semi broadcast nested loop join codegen off
  Stopped after 2 iterations, 6958 ms
  Running case: left semi broadcast nested loop join codegen on
  Stopped after 5 iterations, 3383 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi broadcast nested loop join:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------------------------------------------------------
left semi broadcast nested loop join codegen off           3434           3479          65          6.1         163.7       1.0X
left semi broadcast nested loop join codegen on             672            677           5         31.2          32.1       5.1X
```

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

No.

### How was this patch tested?

Changed existing unit test in `ExistenceJoinSuite.scala` to cover all code paths:
* left semi/anti + empty right side + empty condition
* left semi/anti + non-empty right side + empty condition
* left semi/anti + right side + non-empty condition

Added unit test in `WholeStageCodegenSuite.scala` to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_semi").explain("codegen")
```

Example generated code (`bnlj_doConsume_0` method):
This is for left semi join. The generated code for left anti join is mostly to be same as here, except L55 to be `if (bnlj_findMatchedRow_0 == false) {`.
```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) Project [id#0L AS k1#2L]
+- *(2) BroadcastNestedLoopJoin BuildRight, LeftSemi, ((id#0L + 1) <= k2#6L)
   :- *(2) Range (0, 4, step=1, splits=2)
   +- BroadcastExchange IdentityBroadcastMode, [id=#23]
      +- *(1) Project [id#4L AS k2#6L]
         +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 031 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_findMatchedRow_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */
/* 041 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 042 */
/* 043 */       long bnlj_value_3 = -1L;
/* 044 */
/* 045 */       bnlj_value_3 = bnlj_expr_0_0 + 1L;
/* 046 */
/* 047 */       boolean bnlj_value_2 = false;
/* 048 */       bnlj_value_2 = bnlj_value_3 <= bnlj_value_1;
/* 049 */       if (!(false || !bnlj_value_2))
/* 050 */       {
/* 051 */         bnlj_findMatchedRow_0 = true;
/* 052 */         break;
/* 053 */       }
/* 054 */     }
/* 055 */     if (bnlj_findMatchedRow_0 == true) {
/* 056 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 057 */
/* 058 */       // common sub-expressions
/* 059 */
/* 060 */       range_mutableStateArray_0[3].reset();
/* 061 */
/* 062 */       range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 063 */       append((range_mutableStateArray_0[3].getRow()).copy());
/* 064 */
/* 065 */     }
/* 066 */
/* 067 */   }
/* 068 */
/* 069 */   private void initRange(int idx) {
/* 070 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 071 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 072 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 073 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 074 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 075 */     long partitionEnd;
/* 076 */
/* 077 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 078 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 079 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 080 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 081 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 082 */     } else {
/* 083 */       range_nextIndex_0 = st.longValue();
/* 084 */     }
/* 085 */     range_batchEnd_0 = range_nextIndex_0;
/* 086 */
/* 087 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 088 */     .multiply(step).add(start);
/* 089 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 090 */       partitionEnd = Long.MAX_VALUE;
/* 091 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 092 */       partitionEnd = Long.MIN_VALUE;
/* 093 */     } else {
/* 094 */       partitionEnd = end.longValue();
/* 095 */     }
/* 096 */
/* 097 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 098 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 099 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 100 */     if (range_numElementsTodo_0 < 0) {
/* 101 */       range_numElementsTodo_0 = 0;
/* 102 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 103 */       range_numElementsTodo_0++;
/* 104 */     }
/* 105 */   }
/* 106 */
/* 107 */   protected void processNext() throws java.io.IOException {
/* 108 */     // initialize Range
/* 109 */     if (!range_initRange_0) {
/* 110 */       range_initRange_0 = true;
/* 111 */       initRange(partitionIndex);
/* 112 */     }
/* 113 */
/* 114 */     while (true) {
/* 115 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 116 */         long range_nextBatchTodo_0;
/* 117 */         if (range_numElementsTodo_0 > 1000L) {
/* 118 */           range_nextBatchTodo_0 = 1000L;
/* 119 */           range_numElementsTodo_0 -= 1000L;
/* 120 */         } else {
/* 121 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 122 */           range_numElementsTodo_0 = 0;
/* 123 */           if (range_nextBatchTodo_0 == 0) break;
/* 124 */         }
/* 125 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 126 */       }
/* 127 */
/* 128 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 129 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 130 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 131 */
/* 132 */         bnlj_doConsume_0(range_value_0);
/* 133 */
/* 134 */         if (shouldStop()) {
/* 135 */           range_nextIndex_0 = range_value_0 + 1L;
/* 136 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 137 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 138 */           return;
/* 139 */         }
/* 140 */
/* 141 */       }
/* 142 */       range_nextIndex_0 = range_batchEnd_0;
/* 143 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 144 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 145 */       range_taskContext_0.killTaskIfInterrupted();
/* 146 */     }
/* 147 */   }
/* 148 */
/* 149 */ }
```

Closes #31874 from c21/code-semi-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 07:31:16 +00:00
HyukjinKwon c7bf8adc38 [SPARK-34818][PYTHON][DOCS] Reorder the items in User Guide at PySpark documentation
### What changes were proposed in this pull request?

This PR proposes to reorder the items in User Guide in PySpark documentation in order to place general guides first and advance ones later.

### Why are the changes needed?

For users to more easily follow.

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

Yes, it changes the order in the items in documentation .

### How was this patch tested?

Manually verified the documentation after building:

<img width="768" alt="Screen Shot 2021-03-22 at 2 38 41 PM" src="https://user-images.githubusercontent.com/6477701/111945072-5537d680-8b1c-11eb-9f43-02f3ad63a509.png">

FWIW, the current page: https://spark.apache.org/docs/latest/api/python/user_guide/index.html

Closes #31922 from HyukjinKwon/SPARK-34818.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-22 15:53:39 +09:00
Yuanjian Li 45235ac4bc [SPARK-34748][SS] Create a rule of the analysis logic for streaming write
### What changes were proposed in this pull request?
- Create a new rule `ResolveStreamWrite` for all analysis logic for streaming write.
- Add corresponding logical plans `WriteToStreamStatement` and `WriteToStream`.

### Why are the changes needed?
Currently, the analysis logic for streaming write is mixed in StreamingQueryManager. If we create a specific analyzer rule and separated logical plans, it should be helpful for further extension.

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

### How was this patch tested?
Existing tests.

Closes #31842 from xuanyuanking/SPARK-34748.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 06:39:39 +00:00
Bo Zhang 3bef2dc01a Revert "[SPARK-34757][CORE][DEPLOY] Ignore cache for SNAPSHOT dependencies in spark-submit"
### What changes were proposed in this pull request?

This reverts commit 86ea520320.

### Why are the changes needed?

The test added in the change was flaky.

Closes #31918 from bozhang2820/revert-spark-34757.

Authored-by: Bo Zhang <bo.zhang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-22 15:07:15 +09:00
Kousuke Saruta 0734101bb7 [SPARK-34225][CORE] Don't encode further when a URI form string is passed to addFile or addJar
### What changes were proposed in this pull request?

This PR fixes an issue that `addFile` and `addJar` further encode even though a URI form string is passed.
For example, the following operation will throw exception even though the file exists.
```
sc.addFile("file:/foo/test%20file.txt")
```

Another case is `--files` and `--jars` option when we submit an application.
```
bin/spark-shell --files "/foo/test file.txt"
```
The path above is transformed to URI form [here](ecf4811764/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala (L400)) and passed to `addFile` so the same issue happens.

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

New test.

Closes #31718 from sarutak/fix-uri-encode-double.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-03-22 14:06:41 +09:00
Josh Soref f4de93efb0 [MINOR][SQL] Spelling: filters - PushedFilers
### What changes were proposed in this pull request?
Consistently correct the spelling of `PushedFilters`

### Why are the changes needed?
bersprockets noted that it's wrong

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

Technically, I think it does. Practically, neither Google nor GitHub show anyone using `pushedFilers` outside of forks (or the discussion about fixing it started at https://github.com/apache/spark/pull/30323#issuecomment-725568719)

### How was this patch tested?
None beyond CI in the previous PR

Closes #30678 from jsoref/spelling-filters.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 08:00:12 +03:00
Ruifeng Zheng 47da944f59 [SPARK-34470][ML] VectorSlicer utilize ordering if possible
### What changes were proposed in this pull request?
1, add a new param `sorted` in `slice`;
2, in `VectorSlicer`, set `sorted = true` if input indices are ordered.

### Why are the changes needed?
The input indices of VectorSlicer are probably ordered.
VectorSlicer should use this attribute if possible.

I did a simple test and `sorted = true` maybe about 70% faster than existing `slice`

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

### How was this patch tested?
added testsuite

Closes #31588 from zhengruifeng/vector_slice_for_sorted_indices.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2021-03-22 09:46:53 +08:00
Dongjoon Hyun c5fd94f119 [SPARK-34772][TESTS][FOLLOWUP] Disable a test case using Hive 1.2.1 in Java9+ environment
### What changes were proposed in this pull request?

This PR aims to disable a new test case using Hive 1.2.1 from Java9+ test environment.

### Why are the changes needed?

[HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113) upgraded Datanucleus to 4.x at Hive 2.0. Datanucleus 3.x doesn't support Java9+.

**Java 9+ Environment**
```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] *** 1 TEST FAILED ***
[error] Failed: Total 1, Failed 1, Errors 0, Passed 0
[error] Failed tests:
[error] 	org.apache.spark.sql.hive.HiveSparkSubmitSuite
[error] (hive / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 328 s (05:28), completed Mar 21, 2021, 5:32:39 PM
```

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

Fix the UT in Java9+ environment.

### How was this patch tested?

Manually.

```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] HiveSparkSubmitSuite:
[info] - SPARK-34772: RebaseDateTime loadRebaseRecords should use Spark classloader instead of context !!! CANCELED !!! (26 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:344)
```

Closes #31916 from dongjoon-hyun/SPARK-HiveSparkSubmitSuite.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 17:59:55 -07:00
Dongjoon Hyun 3bc6fe4e77 [SPARK-34809][CORE] Enable spark.hadoopRDD.ignoreEmptySplits by default
### What changes were proposed in this pull request?

This PR aims to enable `spark.hadoopRDD.ignoreEmptySplits` by default for Apache Spark 3.2.0.

### Why are the changes needed?

Although this is a safe improvement, this hasn't been enabled by default to avoid the explicit behavior change. This PR aims to switch the default explicitly in Apache Spark 3.2.0.

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

Yes, the behavior change is documented.

### How was this patch tested?

Pass the existing CIs.

Closes #31909 from dongjoon-hyun/SPARK-34809.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 14:34:02 -07:00
Dongjoon Hyun 3c32b54a0f [SPARK-34811][CORE] Redact fs.s3a.access.key like secret and token
### What changes were proposed in this pull request?

Like we redact secrets and tokens, this PR aims to redact access key.

### Why are the changes needed?

Access key is also worth to hide.

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

This will hide this information from SparkUI (`Spark Properties` and `Hadoop Properties` and logs).

### How was this patch tested?

Pass the newly updated UT.

Closes #31912 from dongjoon-hyun/SPARK-34811.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 14:08:34 -07:00
Zhang, Xingchao 2888d1883e [SPARK-34784][BUILD] Upgrade Jackson to 2.12.2
### What changes were proposed in this pull request?

This pr upgrade Jackson to 2.12.2.
Jackson Release 2.12: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.12

### Why are the changes needed?

Make it easy to upgrade Avro 1.10.2.

```
[error] Caused by: sbt.ForkMain$ForkError: com.fasterxml.jackson.databind.JsonMappingException: Scala module 2.11.4 requires Jackson Databind version >= 2.11.0 and < 2.12.0
[error] 	at com.fasterxml.jackson.module.scala.JacksonModule.setupModule(JacksonModule.scala:61)
[error] 	at com.fasterxml.jackson.module.scala.JacksonModule.setupModule$(JacksonModule.scala:46)
[error] 	at com.fasterxml.jackson.module.scala.DefaultScalaModule.setupModule(DefaultScalaModule.scala:17)
```

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

No.

### How was this patch tested?

Tested with Avro 1.10.2 and Parquet 1.12.0: https://github.com/apache/spark/runs/2157735537

Closes #31878 from xclyfe/SPARK-34784.

Authored-by: Zhang, Xingchao <xingczhang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-21 15:36:38 +08:00
William Hyun c799d049fc [SPARK-34810][TEST] Update PostgreSQL test with the latest results
### What changes were proposed in this pull request?

This PR aims to update `PostgresIntegrationSuite` with the latest results.

### Why are the changes needed?

The latest PostgreSQL jar version is 42.2.19. Since 42.2.9, the test is broken because it returns `0.0` instead of `0.00`.
- https://jdbc.postgresql.org/documentation/changelog.html#version_42.2.19

42.2.9 (2019-12-06)
42.2.10 (2020-01-30)
42.2.11 (2020-03-09)
42.2.12 (2020-03-31)
42.2.13 (2020-06-04)
42.2.14 (2020-06-10)
42.2.15 (2020-08-14)
42.2.16 (2020-08-20)
42.2.17 (2020-10-09)
42.2.18 (2020-10-15)
42.2.19 (2021-02-18)

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

No.

### How was this patch tested?

Pass the CI with the updated test cases.

```
build/sbt -Pdocker-integration-tests 'docker-integration-tests/testOnly org.apache.spark.sql.jdbc.PostgresIntegrationSuite'
```

Closes #31910 from williamhyun/pg.

Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-21 13:36:45 +08:00
Kousuke Saruta 94fd6cb0ce [SPARK-34636][FOLLOWUP][SQL] Fix an incompatible behavior of UnresolvedAttribute.sql
### What changes were proposed in this pull request?

This PR fixes an incompatible behavior introduced by #31754.
The problem is that quoted name parts represented as a string are given to the constructor of `UnresolvedAttribute` which takes single string parameter, `sql` method invocation against the `UnresolvedAttrribute` returns different result than before.

One example is ``` UnresolvedAttribute("`a.b`").sql ```. This  returned `a.b` before but it doesn't now.

See [this duscussion](https://github.com/apache/spark/pull/31754/files#r597181927) for more details.

### Why are the changes needed?

For compatibility.

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

No.

### How was this patch tested?

New assertion.

Closes #31885 from sarutak/followup-SPARK-34636.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-20 14:44:36 -07:00
Yuming Wang 908318f30d [SPARK-28220][SQL] Improve PropagateEmptyRelation to support join with false condition
### What changes were proposed in this pull request?

Improve `PropagateEmptyRelation` to support join with false condition. For example:
```sql
SELECT * FROM t1 LEFT JOIN t2 ON false
```

Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- BroadcastNestedLoopJoin BuildRight, LeftOuter, false
   :- FileScan parquet default.t1[a#4L]
   +- BroadcastExchange IdentityBroadcastMode, [id=#40]
      +- FileScan parquet default.t2[b#5L]
```

After this pr:
```
== Physical Plan ==
*(1) Project [a#4L, null AS b#5L]
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[a#4L]
```

### Why are the changes needed?

Avoid `BroadcastNestedLoopJoin` to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31857 from wangyum/SPARK-28220.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-20 22:57:02 +08:00
Sean Owen ed641fbad6 [MINOR][DOCS][ML] Doc 'mode' as a supported Imputer strategy in Pyspark
### What changes were proposed in this pull request?

Document `mode` as a supported Imputer strategy in Pyspark docs.

### Why are the changes needed?

Support was added in 3.1, and documented in Scala, but some Python docs were missed.

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

No

### How was this patch tested?

Existing tests.

Closes #31883 from srowen/ImputerModeDocs.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-20 01:16:49 -05:00
Kent Yao 2cdedef2a0 [SPARK-34128][SQL] Suppress undesirable TTransportException warnings involved in THRIFT-4805
### What changes were proposed in this pull request?

Since Spark 3.0, the `libthrift` has been bumped up from 0.9.3 to 0.12.0.

Due to THRIFT-4805, The SparkThrift Server will print annoying TExceptions. For example, the current thrift server module test in Github action workflow outputs more than 200MB of data for this error only
```java
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
```

I checked the latest `hive-service-rpc` module in the maven center,  https://mvnrepository.com/artifact/org.apache.hive/hive-service-rpc/3.1.2.  It still uses the 0.9.3 version.

Unfortunately, I tried the newly released `libthrift 0.14.1`(w/o shading it), it breaks the metastore client side.

```scala
java.lang.NoSuchMethodError: org.apache.thrift.transport.TSocket.<init>(Ljava/lang/String;II)V
```
On the Thrift side, they just muted it see https://issues.apache.org/jira/browse/THRIFT-4805

So in this PR, I add a filter to suppress the warning

### Why are the changes needed?

if the log is too large, the Github action might truncate it. We need to reduce useless output.

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

no

### How was this patch tested?

```build/sbt "hive-thriftserver/testOnly *ThriftServerQueryTestSuite" -Phive-thriftserver``` locally

#### before

```java
[info] - count.sql (1 second, 537 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
14:09:53.233 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - group-analytics.sql (4 seconds, 282 milliseconds)

[info] - csv-functions.sql (400 milliseconds)
14:09:24.234 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - datetime-formatting-invalid.sql (349 milliseconds)
14:09:26.544 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - except.sql (2 seconds, 309 milliseconds)
14:09:27.782 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - string-functions.sql (1 second, 237 milliseconds)
14:09:27.835 WARN org.apache.spark.sql.execution.datasources.DataSource: All paths were ignored:

14:09:29.266 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)

```

#### after

```java

[info] - null-propagation.sql (181 milliseconds)
[info] - operators.sql (1 second, 772 milliseconds)
[info] - change-column.sql (241 milliseconds)
[info] - count.sql (1 second, 665 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
[info] - group-analytics.sql (3 seconds, 926 milliseconds)
[info] - inline-table.sql (247 milliseconds)
[info] - comparator.sql (223 milliseconds)
[info] - show-tblproperties.sql (148 milliseconds)
[info] - timezone.sql (105 milliseconds)
[info] - parse-schema-string.sql (193 milliseconds)
```

Closes #31895 from yaooqinn/SPARK-34128-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 21:15:28 -07:00
Ruifeng Zheng f11950f08f [SPARK-32384][CORE] repartitionAndSortWithinPartitions avoid shuffle with same partitioner
### What changes were proposed in this pull request?
avoid unnecessary shuffle if possible

### Why are the changes needed?
avoid unnecessary shuffle.

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

### How was this patch tested?
added testsuites and existing testsuites

Closes #31480 from zhengruifeng/repartitionAndSortWithinPartitions_opt_II.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-03-20 10:29:48 +08:00
Cheng Su 2ff0032e01 [SPARK-34796][SQL] Initialize counter variable for LIMIT code-gen in doProduce()
### What changes were proposed in this pull request?

This PR is to fix the LIMIT code-gen bug in https://issues.apache.org/jira/browse/SPARK-34796, where the counter variable from `BaseLimitExec` is not initialized but used in code-gen. This is because the limit counter variable will be used in upstream operators (LIMIT's child plan, e.g. `ColumnarToRowExec` operator for early termination), but in the same stage, there can be some operators doing the shortcut and not calling `BaseLimitExec`'s `doConsume()`, e.g. [HashJoin.codegenInner](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L402). So if we have query that `LocalLimit - BroadcastHashJoin - FileScan` in the same stage, the whole stage code-gen compilation will be failed.

Here is an example:

```
  test("failed limit query") {
    withTable("left_table", "empty_right_table", "output_table") {
      spark.range(5).toDF("k").write.saveAsTable("left_table")
      spark.range(0).toDF("k").write.saveAsTable("empty_right_table")

      withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
        spark.sql("CREATE TABLE output_table (k INT) USING parquet")
        spark.sql(
          s"""
             |INSERT INTO TABLE output_table
             |SELECT t1.k FROM left_table t1
             |JOIN empty_right_table t2
             |ON t1.k = t2.k
             |LIMIT 3
             |""".stripMargin)
      }
    }
  }
```

Query plan:

```
Execute InsertIntoHadoopFsRelationCommand file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table, false, Parquet, Map(path -> file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table), Append, CatalogTable(
Database: default
Table: output_table
Created Time: Thu Mar 18 21:46:26 PDT 2021
Last Access: UNKNOWN
Created By: Spark 3.2.0-SNAPSHOT
Type: MANAGED
Provider: parquet
Location: file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table
Schema: root
 |-- k: integer (nullable = true)
), org.apache.spark.sql.execution.datasources.InMemoryFileIndexb25d08b, [k]
+- *(3) Project [ansi_cast(k#228L as int) AS k#231]
   +- *(3) GlobalLimit 3
      +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#179]
         +- *(2) LocalLimit 3
            +- *(2) Project [k#228L]
               +- *(2) BroadcastHashJoin [k#228L], [k#229L], Inner, BuildRight, false
                  :- *(2) Filter isnotnull(k#228L)
                  :  +- *(2) ColumnarToRow
                  :     +- FileScan parquet default.left_table[k#228L] Batched: true, DataFilters: [isnotnull(k#228L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
                  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#173]
                     +- *(1) Filter isnotnull(k#229L)
                        +- *(1) ColumnarToRow
                           +- FileScan parquet default.empty_right_table[k#229L] Batched: true, DataFilters: [isnotnull(k#229L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
```

Codegen failure - https://gist.github.com/c21/ea760c75b546d903247582be656d9d66 .

The uninitialized variable `_limit_counter_1` from `LocalLimitExec` is referenced in `ColumnarToRowExec`, but `BroadcastHashJoinExec` does not call `LocalLimitExec.doConsume()` to initialize the counter variable.

The fix is to move the counter variable initialization to `doProduce()`, as in whole stage code-gen framework, `doProduce()` will definitely be called if upstream operators `doProduce()`/`doConsume()` is called.

Note: this only happens in AQE disabled case, because we have an AQE optimization rule [EliminateUnnecessaryJoin](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala#L69) to change the whole query to an empty `LocalRelation` if inner join broadcast side is empty with AQE enabled.

### Why are the changes needed?

Fix query failure.

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

No.

### How was this patch tested?

Added unit test in `SQLQuerySuite.scala`.

Closes #31892 from c21/limit-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:20:52 +09:00
tanel.kiis@gmail.com 620cae098c [SPARK-33122][SQL] Remove redundant aggregates in the Optimzier
### What changes were proposed in this pull request?

Added optimizer rule `RemoveRedundantAggregates`. It removes redundant aggregates from a query plan. A redundant aggregate is an aggregate whose only goal is to keep distinct values, while its parent aggregate would ignore duplicate values.

The affected part of the query plan for TPCDS q87:

Before:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#785]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
            +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                  +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                     +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#724]
                        +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                           +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                              :- ...
```

After:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#751]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#694]
            +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                  :- ...
```

### Why are the changes needed?

Performance improvements - few TPCDS queries have these kinds of duplicate aggregates.

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

No

### How was this patch tested?

UT

Benchmarks (sf=5):

OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Linux 5.8.13-arch1-1
Intel(R) Core(TM) i5-6500 CPU  3.20GHz

| Query | Before  | After | Speedup |
| ------| ------- | ------| ------- |
| q14a | 44s | 44s | 1x |
| q14b | 41s | 41s | 1x |
| q38  | 6.5s | 5.9s | 1.1x |
| q87  | 7.2s | 6.8s | 1.1x |
| q14a-v2.7 | 55s | 53s | 1x |

Closes #30018 from tanelk/SPARK-33122.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:16:39 +09:00
Liang-Chi Hsieh 7a8a600995 [SPARK-34776][SQL] Nested column pruning should not prune Window produced attributes
### What changes were proposed in this pull request?

This patch proposes to fix a bug related to `NestedColumnAliasing`. The root cause is `Window`  doesn't override `producedAttributes` so `NestedColumnAliasing` rule wrongly prune attributes produced by `Window`.

The master and branch-3.1 both have this issue.

### Why are the changes needed?

It is needed to fix a bug of nested column pruning.

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

No

### How was this patch tested?

Unit test.

Closes #31897 from viirya/SPARK-34776.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 11:44:02 -07:00
Dongjoon Hyun 2fa792aa64 [SPARK-34783][K8S] Support remote template files
### What changes were proposed in this pull request?

This PR aims to support remote driver/executor template files.

### Why are the changes needed?

Currently, `KubernetesUtils.loadPodFromTemplate` supports only local files.

With this PR, we can do the following.
```bash
bin/spark-submit \
...
-c spark.kubernetes.driver.podTemplateFile=s3a://dongjoon/driver.yml \
-c spark.kubernetes.executor.podTemplateFile=s3a://dongjoon/executor.yml \
...
```

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

Yes, this is an improvement.

### How was this patch tested?

Manual testing.

Closes #31877 from dongjoon-hyun/SPARK-34783-2.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 08:52:42 -07:00
Max Gekk 089c3b77e1 [SPARK-34793][SQL] Prohibit saving of day-time and year-month intervals
### What changes were proposed in this pull request?
For all built-in datasources, prohibit saving of year-month and day-time intervals that were introduced by SPARK-27793. We plan to support saving of such types at the milestone 2, see SPARK-27790.

### Why are the changes needed?
To improve user experience with Spark SQL, and print nicer error message. Current error message might confuse users:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
21/03/18 22:44:35 ERROR FileFormatWriter: Aborting job 8de402d7-ab69-4dc0-aa8e-14ef06bd2d6b.
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (192.168.1.66 executor driver): org.apache.spark.SparkException: Task failed while writing rows.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.taskFailedWhileWritingRowsError(QueryExecutionErrors.scala:418)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:298)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:211)
	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:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.lang.RuntimeException: Failed to convert value 1 (class of class java.lang.Integer}) with the type of YearMonthIntervalType to JSON.
	at scala.sys.package$.error(package.scala:30)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23(JacksonGenerator.scala:179)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23$adapted(JacksonGenerator.scala:176)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```

### How was this patch tested?
1. Checked nested intervals:
```
scala> spark.range(1).selectExpr("""struct(timestamp'2021-01-02 00:01:02' - timestamp'2021-01-01 00:00:00')""").write.mode("overwrite").parquet("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
scala> Seq(Seq(java.time.Period.ofMonths(1))).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```
2. By running existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2DataFrameSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```

Closes #31884 from MaxGekk/ban-save-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-19 18:47:53 +03:00
Hongyi Zhang 6f89cdfb0c [SPARK-34798][SQL][TESTS] Fix incorrect join condition
### What changes were proposed in this pull request?

join condition 'a.attr == 'c.attr check the reference of  these 2 objects which will always returns false. we need to use === instead

### Why are the changes needed?

Although this join condition always false doesn't break the test but it is not what we expected. We should fix it to avoid future confusing

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

No

### How was this patch tested?

UT

Closes #31890 from opensky142857/SPARK-34798.

Authored-by: Hongyi Zhang <hongyzhang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-19 23:35:15 +08:00
Wenchen Fan 4b4f8e2a25 [SPARK-34558][SQL][FOLLOWUP] Use final Hadoop conf to instantiate FileSystem in SharedState
### What changes were proposed in this pull request?

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

https://github.com/apache/spark/pull/31671 has an unexpected behavior change that it uses a different Hadoop conf (`sparkContext.hadoopConfiguration`) to instantiate `FileSystem`, which is used to qualify the warehouse path. Before https://github.com/apache/spark/pull/31671 , the Hadoop conf to instantiate `FileSystem` is `session.sessionState.newHadoopConf()`.

More specifically, `session.sessionState.newHadoopConf()` has more conf entries:
1. it includes configs from `SharedState.initialConfigs`
2. in includes configs from `sparkContext.conf`

This PR updates `SharedState` to use the final Hadoop conf to instantiate `FileSystem`.

### Why are the changes needed?

fix behavior change

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

yes, the behavior will be the same before https://github.com/apache/spark/pull/31671

### How was this patch tested?

manually check the log of `FileSystem` and verify the passed in configs.

Closes #31868 from cloud-fan/followup.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-19 22:02:15 +08:00