Commit graph

9759 commits

Author SHA1 Message Date
gengjiaan 1597d8fcd4 [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
### What changes were proposed in this pull request?
This PR is related to https://github.com/apache/spark/pull/26656.
https://github.com/apache/spark/pull/26656 only support use FILTER clause on aggregate expression without DISTINCT.
This PR will enhance this feature when one or more DISTINCT aggregate expressions which allows the use of the FILTER clause.
Such as:
```
select sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id) filter (where sex = 'man') from student group by class_id;
select count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student;
select class_id, count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select sum(distinct id), sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select class_id, count(id), count(id) filter (where class_id = 1), sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
```

### Why are the changes needed?
Spark SQL only support use FILTER clause on aggregate expression without DISTINCT.
This PR support Filter expression allows simultaneous use of DISTINCT

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

### How was this patch tested?
Exists and new UT

Closes #29291 from beliefer/support-distinct-with-filter.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 04:41:19 +00:00
Takuya UESHIN 7deb67c28f [SPARK-32160][CORE][PYSPARK][FOLLOWUP] Change the config name to switch allow/disallow SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #29278.
This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.

### Why are the changes needed?

The config name `spark.executor.allowSparkContext` is more reasonable.

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

Yes, the config name is changed.

### How was this patch tested?

Updated tests.

Closes #29340 from ueshin/issues/SPARK-32160/change_config_name.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-04 12:45:06 +09:00
Max Gekk f3b10f526b [SPARK-32290][SQL][FOLLOWUP] Add version for the SQL config spark.sql.optimizeNullAwareAntiJoin
### What changes were proposed in this pull request?
Add the version `3.1.0` for the SQL config `spark.sql.optimizeNullAwareAntiJoin`.

### Why are the changes needed?
To inform users when the config was added, for example on the page http://spark.apache.org/docs/latest/configuration.html.

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

### How was this patch tested?
By compiling and running `./dev/scalastyle`.

Closes #29335 from MaxGekk/leanken-SPARK-32290-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 16:05:54 +00:00
Takeshi Yamamuro c6109ba918 [SPARK-32257][SQL] Reports explicit errors for invalid usage of SET/RESET command
### What changes were proposed in this pull request?

This PR modified the parser code to handle invalid usages of a SET/RESET command.
For example;
```
SET spark.sql.ansi.enabled true
```
The above SQL command does not change the configuration value and it just tries to display the value of the configuration
`spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them. 

Before this PR:
```
scala> sql("SET spark.sql.ansi.enabled true").show(1, -1)
+---------------------------+-----------+
|key                        |value      |
+---------------------------+-----------+
|spark.sql.ansi.enabled true|<undefined>|
+---------------------------+-----------+
```

After this PR:
```
scala> sql("SET spark.sql.ansi.enabled true")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
SET spark.sql.ansi.enabled true
^^^
```

### Why are the changes needed?

For better user-friendly errors.

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

No.

### How was this patch tested?

Added tests in `SparkSqlParserSuite`.

Closes #29146 from maropu/SPARK-32257.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 13:00:07 +00:00
Kent Yao 7f5326c082 [SPARK-32492][SQL] Fulfill missing column meta information COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION for thriftserver client tools
### What changes were proposed in this pull request?

This PR fulfills some missing fields for SparkGetColumnsOperation including COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION

and improve the test coverage.

### Why are the changes needed?

make jdbc tools happier

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

yes,

#### before
![image](https://user-images.githubusercontent.com/8326978/88911764-e78b2180-d290-11ea-8abb-96f137f9c3c4.png)

#### after

![image](https://user-images.githubusercontent.com/8326978/88911709-d04c3400-d290-11ea-90ab-02bda3e628e9.png)

![image](https://user-images.githubusercontent.com/8326978/88912007-39cc4280-d291-11ea-96d6-1ef3abbbddec.png)

### How was this patch tested?

add unit tests

Closes #29303 from yaooqinn/SPARK-32492.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 12:49:34 +00:00
Kent Yao 3deb59d5c2 [SPARK-31709][SQL] Proper base path for database/table location when it is a relative path
### What changes were proposed in this pull request?

Currently, the user home directory is used as the base path for the database and table locations when their locationa are specified with a relative paths, e.g.
```sql
> set spark.sql.warehouse.dir;
spark.sql.warehouse.dir	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/spark-warehouse/
spark-sql> create database loctest location 'loctestdbdir';

spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Owner	kentyao

spark-sql> create table loctest(id int) location 'loctestdbdir';
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	default
Table	loctest
Owner	kentyao
Created Time	Thu May 14 16:29:05 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
The user home is not always warehouse-related, unchangeable in runtime, and shared both by database and table as the parent directory. Meanwhile, we use the table path as the parent directory for relative partition locations.

The config `spark.sql.warehouse.dir` represents `the default location for managed databases and tables`.
For databases, the case above seems not to follow its semantics, because it should use ` `spark.sql.warehouse.dir` as the base path instead.

For tables, it seems to be right but here I suggest enriching the meaning that lets it also be the for external tables with relative paths for locations.

With changes in this PR,

The location of a database will be `warehouseDir/dbpath` when `dbpath` is relative.
The location of a table will be `dbpath/tblpath` when `tblpath` is relative.

### Why are the changes needed?

bugfix and improvement

Firstly, the databases with relative locations should be created under the default location specified by `spark.sql.warehouse.dir`.

Secondly, the external tables with relative paths may also follow this behavior for consistency.

At last, the behavior for database, tables and partitions with relative paths to choose base paths should be the same.

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

Yes, this PR changes the `createDatabase`, `alterDatabase`, `createTable` and `alterTable` APIs and related DDLs. If the LOCATION clause is followed by a relative path, the root path will be `spark.sql.warehouse.dir` for databases, and `spark.sql.warehouse.dir` / `dbPath` for tables.

e.g.

#### after
```sql
spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest
Owner	kentyao
spark-sql> use loctest;
spark-sql> create table loctest(id int) location 'loctest';
20/05/14 18:18:02 WARN InMemoryFileIndex: The directory file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/loctest was not found. Was it deleted very recently?
20/05/14 18:18:02 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
spark-sql> alter table loctest set location 'loctest2'
         > ;
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest2
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
### How was this patch tested?

Add unit tests.

Closes #28527 from yaooqinn/SPARK-31709.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 12:48:22 +00:00
beliefer 42f9ee4c7d [SPARK-24884][SQL] Support regexp function regexp_extract_all
### What changes were proposed in this pull request?
`regexp_extract_all` is a very useful function expanded the capabilities of `regexp_extract`.
There are some description of this function.
```
SELECT regexp_extract('1a 2b 14m', '\d+', 0); -- 1
SELECT regexp_extract_all('1a 2b 14m', '\d+', 0); -- [1, 2, 14]
SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a'
SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm']
```
There are some mainstream database support the syntax.
**Presto:**
https://prestodb.io/docs/current/functions/regexp.html

**Pig:**
https://pig.apache.org/docs/latest/api/org/apache/pig/builtin/REGEX_EXTRACT_ALL.html

Note: This PR pick up the work of https://github.com/apache/spark/pull/21985
### Why are the changes needed?
`regexp_extract_all` is a very useful function and make work easier.

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

### How was this patch tested?
New UT

Closes #27507 from beliefer/support-regexp_extract_all.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 06:03:55 +00:00
Prakhar Jain 7a09e71198 [SPARK-32509][SQL] Ignore unused DPP True Filter in Canonicalization
### What changes were proposed in this pull request?
This PR fixes issues relate to Canonicalization of FileSourceScanExec when it contains unused DPP Filter.

### Why are the changes needed?

As part of PlanDynamicPruningFilter rule, the unused DPP Filter are simply replaced by `DynamicPruningExpression(TrueLiteral)` so that they can be avoided. But these unnecessary`DynamicPruningExpression(TrueLiteral)` partition filter inside the FileSourceScanExec affects the canonicalization of the node and so in many cases, this can prevent ReuseExchange from happening.

This PR fixes this issue by ignoring the unused DPP filter in the `def doCanonicalize` method.

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

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

Closes #29318 from prakharjain09/SPARK-32509_df_reuse.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:26:03 +00:00
Max Gekk fda397d9c8 [SPARK-32510][SQL] Check duplicate nested columns in read from JDBC datasource
### What changes were proposed in this pull request?
Check that there are not duplicate column names on the same level (top level or nested levels) in reading from JDBC datasource. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value:
```
The check takes into account the SQL config `spark.sql.caseSensitive` (`false` by default).

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758, and duplicates in nested structures by https://github.com/apache/spark/pull/29234.

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

### How was this patch tested?
Added new test suite `JdbcNestedDataSourceSuite`.

Closes #29317 from MaxGekk/jdbc-dup-nested-columns.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:20:17 +00:00
Robert (Bobby) Evans 713124d5e3 [SPARK-32274][SQL] Make SQL cache serialization pluggable
### What changes were proposed in this pull request?

Add a config to let users change how SQL/Dataframe data is compressed when cached.

This adds a few new classes/APIs for use with this config.

1. `CachedBatch` is a trait used to tag data that is intended to be cached. It has a few APIs that lets us keep the compression/serialization of the data separate from the metrics about it.
2. `CachedBatchSerializer` provides the APIs that must be implemented to cache data.
    * `convertForCache` is an API that runs a cached spark plan and turns its result into an `RDD[CachedBatch]`.  The actual caching is done outside of this API
   * `buildFilter` is an API that takes a set of predicates and builds a filter function that can be used to filter the `RDD[CachedBatch]` returned by `convertForCache`
   * `decompressColumnar` decompresses an `RDD[CachedBatch]` into an `RDD[ColumnarBatch]` This is only used for a limited set of data types.  These data types may expand in the future.  If they do we can add in a new API with a default value that says which data types this serializer supports.
   * `decompressToRows` decompresses an `RDD[CachedBatch]` into an `RDD[InternalRow]` this API, like `decompressColumnar` decompresses the data in `CachedBatch` but turns it into `InternalRow`s, typically using code generation for performance reasons.

There is also an API that lets you reuse the current filtering based on min/max values. `SimpleMetricsCachedBatch` and `SimpleMetricsCachedBatchSerializer`.

### Why are the changes needed?

This lets users explore different types of compression and compression ratios.

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

This adds in a single config, and exposes some developer API classes described above.

### How was this patch tested?

I ran the unit tests around this and I also did some manual performance tests.  I could find any performance difference between the old and new code, and if there is any it is within error.

Closes #29067 from revans2/pluggable_cache_serializer.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:15:54 +00:00
Wenchen Fan 1c6dff7b5f [SPARK-32083][SQL] AQE coalesce should at least return one partition
### What changes were proposed in this pull request?

This PR updates the AQE framework to at least return one partition during coalescing.

This PR also updates `ShuffleExchangeExec.canChangeNumPartitions` to not coalesce for `SinglePartition`.

### Why are the changes needed?

It's a bit risky to return 0 partitions, as sometimes it's different from empty data. For example, global aggregate will return one result row even if the input table is empty. If there is 0 partition, no task will be run and no result will be returned. More specifically, the global aggregate requires `AllTuples` and we can't coalesce to 0 partitions.

This is not a real bug for now. The global aggregate will be planned as partial and final physical agg nodes. The partial agg will return at least one row, so that the shuffle still have data. But it's better to fix this issue to avoid potential bugs in the future.

According to https://github.com/apache/spark/pull/28916, this change also fix some perf problems.

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

no

### How was this patch tested?

updated test.

Closes #29307 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 14:20:20 +00:00
Yuanjian Li 354313b6bc [SPARK-31894][SS][FOLLOW-UP] Rephrase the config doc
### What changes were proposed in this pull request?
Address comment in https://github.com/apache/spark/pull/28707#discussion_r461102749

### Why are the changes needed?
Hide the implementation details in the config doc.

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

### How was this patch tested?
Document only.

Closes #29315 from xuanyuanking/SPARK-31894-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 14:16:41 +00:00
Kent Yao f4800406a4 [SPARK-32406][SQL][FOLLOWUP] Make RESET fail against static and core configs
### What changes were proposed in this pull request?

This followup addresses comments from https://github.com/apache/spark/pull/29202#discussion_r462054784

1. make RESET static SQL configs/spark core configs fail as same as the SET command. Not that, for core ones, they have to be pre-registered, otherwise, they are still able to be SET/RESET

2. add test cases for configurations w/ optional default values

### Why are the changes needed?

behavior change with suggestions from PMCs

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

Yes, RESET will fail after this PR, before it just does nothing because the static ones are static.

### How was this patch tested?

add more tests.

Closes #29297 from yaooqinn/SPARK-32406-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 22:11:05 +09:00
Takuya UESHIN 8014b0b5d6 [SPARK-32160][CORE][PYSPARK] Add a config to switch allow/disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #28986.
This PR adds a config to switch allow/disallow to create `SparkContext` in executors.

- `spark.driver.allowSparkContextInExecutors`

### Why are the changes needed?

Some users or libraries actually create `SparkContext` in executors.
We shouldn't break their workloads.

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

Yes, users will be able to create `SparkContext` in executors with the config enabled.

### How was this patch tested?

More tests are added.

Closes #29278 from ueshin/issues/SPARK-32160/add_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 17:28:35 +09:00
Cheng Su ae82768c13 [SPARK-32421][SQL] Add code-gen for shuffled hash join
### What changes were proposed in this pull request?

Adding codegen for shuffled hash join. Shuffled hash join codegen is very similar to broadcast hash join codegen. So most of code change is to refactor existing codegen in `BroadcastHashJoinExec` to `HashJoin`.

Example codegen for query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153):

```
  def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 20
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000",
      SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"id as k1")
        val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2")
        val df = df1.join(df2, col("k1") === col("k2"))
        assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined)
        df.noop()
      }
    }
  }
```

Shuffled hash join codegen:

```
== Subtree 3 / 3 (maxMethodCodeSize:113; maxConstantPoolSize:126(0.19% used); numInnerClasses:0) ==
*(3) ShuffledHashJoin [k1#2L], [k2#6L], Inner, BuildRight
:- *(1) Project [id#0L AS k1#2L]
:  +- *(1) Range (0, 4194304, step=1, splits=1)
+- *(2) Project [(id#4L * 3) AS k2#6L]
   +- *(2) Range (0, 1398101, step=1, splits=1)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage3(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=3
/* 006 */ final class GeneratedIteratorForCodegenStage3 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.execution.joins.HashedRelation shj_relation_0;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] shj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 012 */
/* 013 */   public GeneratedIteratorForCodegenStage3(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input_0 = inputs[0];
/* 021 */     shj_relation_0 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[0] /* plan */).buildHashedRelation(inputs[1]);
/* 022 */     shj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 023 */
/* 024 */   }
/* 025 */
/* 026 */   private void shj_doConsume_0(InternalRow inputadapter_row_0, long shj_expr_0_0) throws java.io.IOException {
/* 027 */     // generate join key for stream side
/* 028 */
/* 029 */     // find matches from HashRelation
/* 030 */     scala.collection.Iterator shj_matches_0 = false ?
/* 031 */     null : (scala.collection.Iterator)shj_relation_0.get(shj_expr_0_0);
/* 032 */     if (shj_matches_0 != null) {
/* 033 */       while (shj_matches_0.hasNext()) {
/* 034 */         UnsafeRow shj_matched_0 = (UnsafeRow) shj_matches_0.next();
/* 035 */         {
/* 036 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[1] /* numOutputRows */).add(1);
/* 037 */
/* 038 */           long shj_value_1 = shj_matched_0.getLong(0);
/* 039 */           shj_mutableStateArray_0[0].reset();
/* 040 */
/* 041 */           shj_mutableStateArray_0[0].write(0, shj_expr_0_0);
/* 042 */
/* 043 */           shj_mutableStateArray_0[0].write(1, shj_value_1);
/* 044 */           append((shj_mutableStateArray_0[0].getRow()).copy());
/* 045 */
/* 046 */         }
/* 047 */       }
/* 048 */     }
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     while ( inputadapter_input_0.hasNext()) {
/* 054 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 055 */
/* 056 */       long inputadapter_value_0 = inputadapter_row_0.getLong(0);
/* 057 */
/* 058 */       shj_doConsume_0(inputadapter_row_0, inputadapter_value_0);
/* 059 */       if (shouldStop()) return;
/* 060 */     }
/* 061 */   }
/* 062 */
/* 063 */ }
```

Broadcast hash join codegen for the same query (for reference here):

```
== Subtree 2 / 2 (maxMethodCodeSize:280; maxConstantPoolSize:218(0.33% used); numInnerClasses:0) ==
*(2) BroadcastHashJoin [k1#2L], [k2#6L], Inner, BuildRight, false
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4194304, step=1, splits=1)
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#22]
   +- *(1) Project [(id#4L * 3) AS k2#6L]
      +- *(1) Range (0, 1398101, step=1, splits=1)

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 org.apache.spark.sql.execution.joins.LongHashedRelation bhj_relation_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 */
/* 032 */     bhj_relation_0 = ((org.apache.spark.sql.execution.joins.LongHashedRelation) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcast */).value()).asReadOnlyCopy();
/* 033 */     incPeakExecutionMemory(bhj_relation_0.estimatedSize());
/* 034 */
/* 035 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 036 */
/* 037 */   }
/* 038 */
/* 039 */   private void initRange(int idx) {
/* 040 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 041 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 042 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4194304L);
/* 043 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 044 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 045 */     long partitionEnd;
/* 046 */
/* 047 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 048 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 049 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 050 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 051 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 052 */     } else {
/* 053 */       range_nextIndex_0 = st.longValue();
/* 054 */     }
/* 055 */     range_batchEnd_0 = range_nextIndex_0;
/* 056 */
/* 057 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 058 */     .multiply(step).add(start);
/* 059 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 060 */       partitionEnd = Long.MAX_VALUE;
/* 061 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 062 */       partitionEnd = Long.MIN_VALUE;
/* 063 */     } else {
/* 064 */       partitionEnd = end.longValue();
/* 065 */     }
/* 066 */
/* 067 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 068 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 069 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 070 */     if (range_numElementsTodo_0 < 0) {
/* 071 */       range_numElementsTodo_0 = 0;
/* 072 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 073 */       range_numElementsTodo_0++;
/* 074 */     }
/* 075 */   }
/* 076 */
/* 077 */   private void bhj_doConsume_0(long bhj_expr_0_0) throws java.io.IOException {
/* 078 */     // generate join key for stream side
/* 079 */
/* 080 */     // find matches from HashedRelation
/* 081 */     UnsafeRow bhj_matched_0 = false ? null: (UnsafeRow)bhj_relation_0.getValue(bhj_expr_0_0);
/* 082 */     if (bhj_matched_0 != null) {
/* 083 */       {
/* 084 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 085 */
/* 086 */         long bhj_value_2 = bhj_matched_0.getLong(0);
/* 087 */         range_mutableStateArray_0[3].reset();
/* 088 */
/* 089 */         range_mutableStateArray_0[3].write(0, bhj_expr_0_0);
/* 090 */
/* 091 */         range_mutableStateArray_0[3].write(1, bhj_value_2);
/* 092 */         append((range_mutableStateArray_0[3].getRow()));
/* 093 */
/* 094 */       }
/* 095 */     }
/* 096 */
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     // initialize Range
/* 101 */     if (!range_initRange_0) {
/* 102 */       range_initRange_0 = true;
/* 103 */       initRange(partitionIndex);
/* 104 */     }
/* 105 */
/* 106 */     while (true) {
/* 107 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 108 */         long range_nextBatchTodo_0;
/* 109 */         if (range_numElementsTodo_0 > 1000L) {
/* 110 */           range_nextBatchTodo_0 = 1000L;
/* 111 */           range_numElementsTodo_0 -= 1000L;
/* 112 */         } else {
/* 113 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 114 */           range_numElementsTodo_0 = 0;
/* 115 */           if (range_nextBatchTodo_0 == 0) break;
/* 116 */         }
/* 117 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 118 */       }
/* 119 */
/* 120 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 121 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 122 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 123 */
/* 124 */         bhj_doConsume_0(range_value_0);
/* 125 */
/* 126 */         if (shouldStop()) {
/* 127 */           range_nextIndex_0 = range_value_0 + 1L;
/* 128 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 129 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 130 */           return;
/* 131 */         }
/* 132 */
/* 133 */       }
/* 134 */       range_nextIndex_0 = range_batchEnd_0;
/* 135 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 136 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 137 */       range_taskContext_0.killTaskIfInterrupted();
/* 138 */     }
/* 139 */   }
/* 140 */
/* 141 */ }
```

### Why are the changes needed?

Codegen shuffled hash join can help save CPU cost. We added shuffled hash join codegen internally in our fork, and seeing obvious improvement in benchmark compared to current non-codegen code path.

Test example query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153), seeing 30% wall clock time improvement compared to existing non-codegen code path:

Enable shuffled hash join code-gen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1358 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2323 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    649            679          43          6.5         154.7       1.0X
shuffle hash join wholestage on                     436            465          45          9.6         103.9       1.5X
```

Disable shuffled hash join codegen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1345 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2967 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    646            673          37          6.5         154.1       1.0X
shuffle hash join wholestage on                     549            594          47          7.6         130.9       1.2X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite`.

Closes #29277 from c21/codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 05:51:57 +00:00
Takeshi Yamamuro 30e3042dc5 [SPARK-32488][SQL] Use @parser::members and @lexer::members to avoid generating unused code
### What changes were proposed in this pull request?

This PR aims to update `SqlBse.g4` for avoiding generating unused code.
Currently, ANTLR generates unused methods and variables; `isValidDecimal` and `isHint` are only used in the generated lexer. This PR changed the code to use `parser::members` and `lexer::members` to avoid it.

### Why are the changes needed?

To reduce unnecessary code.

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

No.

### How was this patch tested?

Existing tests.

Closes #29296 from maropu/UpdateSqlBase.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 07:51:27 +00:00
Kent Yao 510a1656e6 [SPARK-32412][SQL] Unify error handling for spark thrift server operations
### What changes were proposed in this pull request?

Log error/warn message only once at the server-side for both sync and async modes

### Why are the changes needed?

In b151194299 we make the error logging for  SparkExecuteStatementOperation with `runInBackground=true` not duplicated, but the operations with runInBackground=false and other metadata operation still will be log twice which happened in the operation's `runInternal` method and ThriftCLIService.

In this PR, I propose to reflect the logic to get a unified error handling approach.

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

Yes, when spark.sql.hive.thriftServer.async=false and people call sync APIs the error message will be logged only once at server-side.
### How was this patch tested?

locally verified the result in target/unit-test.log

add unit tests.

Closes #29204 from yaooqinn/SPARK-32412.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:44:13 +00:00
HyukjinKwon e1d7321034 [SPARK-32478][R][SQL] Error message to show the schema mismatch in gapply with Arrow vectorization
### What changes were proposed in this pull request?

This PR proposes to:

1. Fix the error message when the output schema is misbatched with R DataFrame from the given function. For example,

    ```R
    df <- createDataFrame(list(list(a=1L, b="2")))
    count(gapply(df, "a", function(key, group) { group }, structType("a int, b int")))
    ```

    **Before:**

    ```
    Error in handleErrors(returnStatus, conn) :
      ...
      java.lang.UnsupportedOperationException
	    ...
    ```

    **After:**

    ```
    Error in handleErrors(returnStatus, conn) :
     ...
     java.lang.AssertionError: assertion failed: Invalid schema from gapply: expected IntegerType, IntegerType, got IntegerType, StringType
        ...
    ```

2. Update documentation about the schema matching for `gapply` and `dapply`.

### Why are the changes needed?

To show which schema is not matched, and let users know what's going on.

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

Yes, error message is updated as above, and documentation is updated.

### How was this patch tested?

Manually tested and unitttests were added.

Closes #29283 from HyukjinKwon/r-vectorized-error.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 15:16:02 +09:00
Max Gekk 99a855575c [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources
### What changes were proposed in this pull request?
When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema:
```

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758.

### Does this PR introduce _any_ user-facing change?
Yes. For the example from SPARK-32431:

ORC:
```scala
java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3
	at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61)
	at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323)
```

JSON:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

Parquet:
```scala
+------------+
|StructColumn|
+------------+
|     [0,, 1]|
+------------+
```

Avro:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

After the changes, Parquet, ORC, JSON and Avro output the same error:
```scala
Found duplicate column(s) in the data schema: `camelcase`;
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`;
	at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67)
```

### How was this patch tested?
Run modified test suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite"
$ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*"
```
and added new UT to `SchemaUtilsSuite`.

Closes #29234 from MaxGekk/nested-case-insensitive-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:05:55 +00:00
Wenchen Fan a025a89f4e [SPARK-32332][SQL] Support columnar exchanges
### What changes were proposed in this pull request?

This PR adds abstract classes for shuffle and broadcast, so that users can provide their columnar implementations.

This PR updates several places to use the abstract exchange classes, and also update `AdaptiveSparkPlanExec` so that the columnar rules can see exchange nodes.

This is an alternative of https://github.com/apache/spark/pull/29134 .
Close https://github.com/apache/spark/pull/29134

### Why are the changes needed?

To allow columnar exchanges.

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

no

### How was this patch tested?

new tests

Closes #29262 from cloud-fan/columnar.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 14:21:47 -05:00
Max Gekk d897825d2d [SPARK-32346][SQL] Support filters pushdown in Avro datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in Avro datasource V1 and V2.
1. Added new SQL config `spark.sql.avro.filterPushdown.enabled` to control filters pushdown to Avro datasource. It is on by default.
2. Renamed `CSVFilters` to `OrderedFilters`.
3. `OrderedFilters` is used in `AvroFileFormat` (DSv1) and in `AvroPartitionReaderFactory` (DSv2)
4. Modified `AvroDeserializer` to return None from the `deserialize` method when pushdown filters return `false`.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **2** times on JDK 11:
```
OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                        9614           9669          54          0.1        9614.1       1.0X
pushdown disabled                                 10077          10141          66          0.1       10077.2       1.0X
w/ filters                                         4681           4713          29          0.2        4681.5       2.1X
```

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

### How was this patch tested?
- Added UT to `AvroCatalystDataConversionSuite` and `AvroSuite`
- Re-running `AvroReadBenchmark` using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

and `./dev/run-benchmarks`:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
  ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #29145 from MaxGekk/avro-filters-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-30 01:37:42 +08:00
LantaoJin 26e6574d58 [SPARK-32283][CORE] Kryo should support multiple user registrators
### What changes were proposed in this pull request?
`spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala`

### Why are the changes needed?
In previous Spark version (2.x), it supported multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it doesn't work in 3.0. It's should be a regression.

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

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

Closes #29123 from LantaoJin/SPARK-32283.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:58:03 +00:00
Terry Kim 45b7212fd3 [SPARK-32401][SQL] Migrate function related commands to use UnresolvedFunc to resolve function identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following function related commands to use `UnresolvedFunc` to resolve function identifier:
- DROP FUNCTION
- DESCRIBE FUNCTION
- SHOW FUNCTIONS

`DropFunctionStatement`, `DescribeFunctionStatement` and `ShowFunctionsStatement` logical plans are replaced with `DropFunction`, `DescribeFunction` and `ShowFunctions` logical plans respectively, and each contains `UnresolvedFunc` as its child so that it can be resolved in `Analyzer`.

### Why are the changes needed?

Migrating to the new resolution framework, which resolves `UnresolvedFunc` in `Analyzer`.

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

The message of exception thrown when a catalog is resolved to v2 has been merged to:
`function is only supported in v1 catalog`

Previously, it printed out the command used. E.g.,:
`CREATE FUNCTION is only supported in v1 catalog`

### How was this patch tested?

Updated existing tests.

Closes #29198 from imback82/function_framework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:55:48 +00:00
Max Gekk b2180c0950 [SPARK-32471][SQL][DOCS][TESTS][PYTHON][SS] Describe JSON option allowNonNumericNumbers
### What changes were proposed in this pull request?
1. Describe the JSON option `allowNonNumericNumbers` which is used in read
2. Add new test cases for allowed JSON field values: NaN, +INF, +Infinity, Infinity, -INF and -Infinity

### Why are the changes needed?
To improve UX with Spark SQL and to provide users full info about the supported option.

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

### How was this patch tested?
Added new test to `JsonParsingOptionsSuite`

Closes #29275 from MaxGekk/allowNonNumericNumbers-doc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-29 12:14:13 +09:00
Michael Munday a3d80564ad [SPARK-32458][SQL][TESTS] Fix incorrectly sized row value reads
### What changes were proposed in this pull request?
Updates to tests to use correctly sized `getInt` or `getLong` calls.

### Why are the changes needed?
The reads were incorrectly sized (i.e. `putLong` paired with `getInt` and `putInt` paired with `getLong`). This causes test failures on big-endian systems.

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

### How was this patch tested?
Tests were run on a big-endian system (s390x). This change is unlikely to have any practical effect on little-endian systems.

Closes #29258 from mundaym/fix-row.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 10:36:20 -07:00
Max Gekk c28da672f8 [SPARK-32382][SQL] Override table renaming in JDBC dialects
### What changes were proposed in this pull request?
Override the default implementation of `JdbcDialect.renameTable()`:
```scala
s"ALTER TABLE $oldTable RENAME TO $newTable"
```
in the following JDBC dialects according to official documentation:
- DB2
- Derby
- MS SQL Server
- Teradata

Other dialects follow the default implementation:
- MySQL: https://dev.mysql.com/doc/refman/8.0/en/alter-table.html
- Oracle: https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/ALTER-TABLE.html#GUID-552E7373-BF93-477D-9DA3-B2C9386F2877
- PostgreSQL: https://www.postgresql.org/docs/12/sql-altertable.html

### Why are the changes needed?
To have correct implementation of table renaming for all supported JDBC dialects.

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

### How was this patch tested?
Manually

Closes #29237 from MaxGekk/jdbc-rename-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:34:10 +00:00
yi.wu ca1ecf7f9f [SPARK-32459][SQL] Support WrappedArray as customCollectionCls in MapObjects
### What changes were proposed in this pull request?

This PR supports `WrappedArray` as `customCollectionCls` in `MapObjects`.

### Why are the changes needed?

This helps fix the regression caused by SPARK-31826.  For the following test, it can pass in branch-3.0 but fail in master branch:

```scala
test("WrappedArray") {
    val myUdf = udf((a: WrappedArray[Int]) =>
      WrappedArray.make[Int](Array(a.head + 99)))
    checkAnswer(Seq(Array(1))
      .toDF("col")
      .select(myUdf(Column("col"))),
      Row(ArrayBuffer(100)))
  }
```

In SPARK-31826, we've changed the catalyst-to-scala converter from `CatalystTypeConverters` to `ExpressionEncoder.deserializer`. However, `CatalystTypeConverters` supports `WrappedArray` while `ExpressionEncoder.deserializer` doesn't.

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

No,  SPARK-31826 is merged into master and branch-3.1, which haven't been released.

### How was this patch tested?

Added a new test for `WrappedArray` in `UDFSuite`; Also updated `ObjectExpressionsSuite` for `MapObjects`.

Closes #29261 from Ngone51/fix-wrappedarray.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:24:15 +00:00
xuewei.linxuewei 12b9787a7f [SPARK-32290][SQL] SingleColumn Null Aware Anti Join Optimize
### What changes were proposed in this pull request?
Normally, a Null aware anti join will be planed into BroadcastNestedLoopJoin which is very time consuming, for instance, in TPCH Query 16.

```
select
    p_brand,
    p_type,
    p_size,
    count(distinct ps_suppkey) as supplier_cnt
from
    partsupp,
    part
where
    p_partkey = ps_partkey
    and p_brand <> 'Brand#45'
    and p_type not like 'MEDIUM POLISHED%'
    and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
    and ps_suppkey not in (
        select
            s_suppkey
        from
            supplier
        where
            s_comment like '%Customer%Complaints%'
    )
group by
    p_brand,
    p_type,
    p_size
order by
    supplier_cnt desc,
    p_brand,
    p_type,
    p_size
```

In above query, will planed into

LeftAnti
    condition Or((ps_suppkey=s_suppkey), IsNull(ps_suppkey=s_suppkey))

Inside BroadcastNestedLoopJoinExec will perform O(M\*N), BUT if there is only single column in NAAJ, we can always change buildSide into a HashSet, and streamedSide just need to lookup in the HashSet, then the calculation will be optimized into O(M).

But this optimize is only targeting on null aware anti join with single column case, because multi-column support is much more complicated, we might be able to support multi-column in future.
After apply this patch, the TPCH Query 16 performance decrease from 41mins to 30s

The semantic of null-aware anti join is:

![image](https://user-images.githubusercontent.com/17242071/88077041-66a39a00-cbad-11ea-8fb6-c235c4d219b4.png)

### Why are the changes needed?
TPCH is a common benchmark for distributed compute engine, all other 21 Query works fine on Spark, except for Query 16, apply this patch will make Spark more competitive among all these popular engine. BTW, this patch has restricted rules and only apply on NAAJ Single Column case, which is safe enough.

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

### How was this patch tested?
1. SQLQueryTestSuite with NOT IN keyword SQL, add CONFIG_DIM with spark.sql.optimizeNullAwareAntiJoin on and off
2. added case in org.apache.spark.sql.JoinSuite.
3. added case in org.apache.spark.sql.SubquerySuite.
3. Compare performance before and after applying this patch against TPCH Query 16.
4. config combination against e2e test with following

```
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "sspark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "true"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "true"
)
```

Closes #29104 from leanken/leanken-SPARK-32290.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 04:42:15 +00:00
Frank Yin 8323c8eb56 [SPARK-32059][SQL] Allow nested schema pruning thru window/sort plans
### What changes were proposed in this pull request?
This PR is intended to solve schema pruning not working with window functions, as described in SPARK-32059. It also solved schema pruning not working with `Sort`. It also generalizes with `Project->Filter->[any node can be pruned]`.

### Why are the changes needed?
This is needed because of performance issues with nested structures with querying using window functions as well as sorting.

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

### How was this patch tested?
Introduced two tests: 1) optimizer planning level 2) end-to-end tests with SQL queries.

Closes #28898 from frankyin-factual/master.

Authored-by: Frank Yin <frank@factual.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-28 10:00:21 +09:00
HyukjinKwon c1140661bf [SPARK-32443][CORE] Use POSIX-compatible command -v in testCommandAvailable
### What changes were proposed in this pull request?

This PR aims to use `command -v` in non-Window operating systems instead of executing the given command.

### Why are the changes needed?

1. `command` is POSIX-compatible
    - **POSIX.1-2017**:  https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html
2. `command` is faster and safer than the direct execution
    - `command` doesn't invoke another process.
```scala
scala> sys.process.Process("ls").run().exitValue()
LICENSE
NOTICE
bin
doc
lib
man
res1: Int = 0
```

3. The existing way behaves inconsistently.
    - `rm` cannot be checked.

**AS-IS**
```scala
scala> sys.process.Process("rm").run().exitValue()
usage: rm [-f | -i] [-dPRrvW] file ...
       unlink file
res0: Int = 64
```

**TO-BE**
```
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue()
/bin/ls
val res1: Int = 0
```

4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following.
```scala
$ bin/scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> sys.process.Process("cat").run().exitValue() // hang here.
```

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

No. Although this is inside `main` source directory, this is used for testing purpose.

```
$ git grep testCommandAvailable | grep -v 'def testCommandAvailable'
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("wc"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(!TestUtils.testCommandAvailable("some_nonexistent_command"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:  private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec)
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:    if (TestUtils.testCommandAvailable(pythonExec)) {
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("python"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("echo | sed"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
```

### How was this patch tested?

- **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test.
- **Scala 2.13**: Do the following manually. It should pass instead of `hang`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite
...
Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29241 from dongjoon-hyun/SPARK-32443.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-27 12:02:43 -07:00
Kent Yao d315ebf3a7 [SPARK-32424][SQL] Fix silent data change for timestamp parsing if overflow happens
### What changes were proposed in this pull request?

When using `Seconds.toMicros` API to convert epoch seconds to microseconds,

```scala
 /**
     * Equivalent to
     * {link #convert(long, TimeUnit) MICROSECONDS.convert(duration, this)}.
     * param duration the duration
     * return the converted duration,
     * or {code Long.MIN_VALUE} if conversion would negatively
     * overflow, or {code Long.MAX_VALUE} if it would positively overflow.
     */
```
This PR change it to `Math.multiplyExact(epochSeconds, MICROS_PER_SECOND)`

### Why are the changes needed?

fix silent data change between 3.x and 2.x
```
 ~/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200722   bin/spark-sql -S -e "select to_timestamp('300000', 'y');"
+294247-01-10 12:00:54.775807
```
```
 kentyaohulk  ~/Downloads/spark/spark-2.4.5-bin-hadoop2.7  bin/spark-sql -S  -e "select to_timestamp('300000', 'y');"
284550-10-19 15:58:1010.448384
```

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

Yes, we will raise `ArithmeticException` instead of giving the wrong answer if overflow.

### How was this patch tested?

add unit test

Closes #29220 from yaooqinn/SPARK-32424.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:03:14 +00:00
Cheng Su 548b7db345 [SPARK-32420][SQL] Add handling for unique key in non-codegen hash join
### What changes were proposed in this pull request?

`HashRelation` has two separate code paths for unique key look up and non-unique key look up E.g. in its subclass [`UnsafeHashedRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala#L144-L177), unique key look up is more efficient as it does not have e.g. extra `Iterator[UnsafeRow].hasNext()/next()` overhead per row.

`BroadcastHashJoinExec` has handled unique key vs non-unique key separately in [code-gen path](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala#L289-L321). But the non-codegen path for broadcast hash join and shuffled hash join do not separate it yet, so adding the support here.

### Why are the changes needed?

Shuffled hash join and non-codegen broadcast hash join still rely on this code path for execution. So this PR will help save CPU for executing this two type of join. Adding codegen for shuffled hash join would be a different topic and I will add it in https://issues.apache.org/jira/browse/SPARK-32421 .

Ran the same query as [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153-L167), with enabling and disabling this feature. Verified 20% wall clock time improvement (switch control and test group order as well to verify the improvement to not be the noise).

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4039 ms
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 2898 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ off                707            808          81          5.9         168.6       1.0X
shuffle hash join unique key SHJ on                 547            580          50          7.7         130.4       1.3X
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 3333 ms
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4268 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ on                 565            667          60          7.4         134.8       1.0X
shuffle hash join unique key SHJ off                774            854          85          5.4         184.4       0.7X
```

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

No.

### How was this patch tested?

* Added test in `OuterJoinSuite` to cover left outer and right outer join.
* Added test in `ExistenceJoinSuite` to cover left semi join, and existence join.
* [Existing `joinSuite` already covered inner join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala#L182)
* [Existing `ExistenceJoinSuite` already covered left anti join, and existence join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala#L228)

Closes #29216 from c21/unique-key.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:01:03 +00:00
SaurabhChawla 99f33ec30f [SPARK-32234][FOLLOWUP][SQL] Update the description of utility method
### What changes were proposed in this pull request?
As the part of this PR https://github.com/apache/spark/pull/29045 added the helper method. This PR is the FOLLOWUP PR to update the description of helper method.

### Why are the changes needed?
For better readability and understanding of the code

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

### How was this patch tested?
Since its only change of updating the description , So ran the Spark shell

Closes #29232 from SaurabhChawla100/SPARK-32234-Desc.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 08:14:02 +00:00
Cheng Su 01cf8a4ce8 [SPARK-32383][SQL] Preserve hash join (BHJ and SHJ) stream side ordering
### What changes were proposed in this pull request?

Currently `BroadcastHashJoinExec` and `ShuffledHashJoinExec` do not preserve children output ordering information (inherit from `SparkPlan.outputOrdering`, which is Nil). This can add unnecessary sort in complex queries involved multiple joins.

Example:

```
withSQLConf(
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") {
      val df1 = spark.range(100).select($"id".as("k1"))
      val df2 = spark.range(100).select($"id".as("k2"))
      val df3 = spark.range(3).select($"id".as("k3"))
      val df4 = spark.range(100).select($"id".as("k4"))
      val plan = df1.join(df2, $"k1" === $"k2")
        .join(df3, $"k1" === $"k3")
        .join(df4, $"k1" === $"k4")
        .queryExecution
        .executedPlan
}
```

Current physical plan (extra sort on `k1` before top sort merge join):

```
*(9) SortMergeJoin [k1#220L], [k4#232L], Inner
:- *(6) Sort [k1#220L ASC NULLS FIRST], false, 0
:  +- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight
:     :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner
:     :  :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0
:     :  :  +- Exchange hashpartitioning(k1#220L, 5), true, [id=#128]
:     :  :     +- *(1) Project [id#218L AS k1#220L]
:     :  :        +- *(1) Range (0, 100, step=1, splits=2)
:     :  +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0
:     :     +- Exchange hashpartitioning(k2#224L, 5), true, [id=#134]
:     :        +- *(3) Project [id#222L AS k2#224L]
:     :           +- *(3) Range (0, 100, step=1, splits=2)
:     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#141]
:        +- *(5) Project [id#226L AS k3#228L]
:           +- *(5) Range (0, 3, step=1, splits=2)
+- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k4#232L, 5), true, [id=#148]
      +- *(7) Project [id#230L AS k4#232L]
         +- *(7) Range (0, 100, step=1, splits=2)
```

Ideal physical plan (no extra sort on `k1` before top sort merge join):

```
*(9) SortMergeJoin [k1#220L], [k4#232L], Inner
:- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight
:  :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner
:  :  :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0
:  :  :  +- Exchange hashpartitioning(k1#220L, 5), true, [id=#127]
:  :  :     +- *(1) Project [id#218L AS k1#220L]
:  :  :        +- *(1) Range (0, 100, step=1, splits=2)
:  :  +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0
:  :     +- Exchange hashpartitioning(k2#224L, 5), true, [id=#133]
:  :        +- *(3) Project [id#222L AS k2#224L]
:  :           +- *(3) Range (0, 100, step=1, splits=2)
:  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#140]
:     +- *(5) Project [id#226L AS k3#228L]
:        +- *(5) Range (0, 3, step=1, splits=2)
+- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k4#232L, 5), true, [id=#146]
      +- *(7) Project [id#230L AS k4#232L]
         +- *(7) Range (0, 100, step=1, splits=2)
```

### Why are the changes needed?

To avoid unnecessary sort in query, and it has most impact when users read sorted bucketed table.
Though the unnecessary sort is operating on already sorted data, it would have obvious negative impact on IO and query run time if the data is large and external sorting happens.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29181 from c21/ordering.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 04:51:32 +00:00
sychen be9f03dc71 [SPARK-32426][SQL] ui shows sql after variable substitution
### What changes were proposed in this pull request?
When submitting sql with variables, the sql displayed by ui is not replaced by variables.

### Why are the changes needed?
See the final executed sql in ui

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

### How was this patch tested?
manual test

Closes #29221 from cxzl25/SPARK-32426.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 03:30:01 -07:00
HyukjinKwon 277a4063ef [SPARK-32422][SQL][TESTS] Use python3 executable instead of python3.6 in IntegratedUDFTestUtils
### What changes were proposed in this pull request?

This PR uses `python3` instead of `python3.6` executable as a fallback in `IntegratedUDFTestUtils`.

### Why are the changes needed?

Currently, GitHub Actions skips pandas UDFs. Python 3.8 is installed explicitly but somehow `python3.6` looks available in GitHub Actions build environment by default.

```
[info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!!
...
[info] - udf/postgreSQL/udf-select_having.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!!
...
```

It was chosen as `python3.6` for Jenkins to pick one Python explicitly; however, looks we're already using `python3` here and there.

It will also reduce the overhead to fix when we deprecate or drop Python versions.

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

No, dev-only.

### How was this patch tested?

It should be tested in Jenkins and GitHub Actions environments here.

Closes #29217 from HyukjinKwon/SPARK-32422.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 03:06:45 -07:00
Andy Grove 64a01c0a55 [SPARK-32430][SQL] Extend SparkSessionExtensions to inject rules into AQE query stage preparation
### What changes were proposed in this pull request?

Provide a generic mechanism for plugins to inject rules into the AQE "query prep" stage that happens before query stage creation.

This goes along with https://issues.apache.org/jira/browse/SPARK-32332 where the current AQE implementation doesn't allow for users to properly extend it for columnar processing.

### Why are the changes needed?

The issue here is that we create new query stages but we do not have access to the parent plan of the new query stage so certain things can not be determined because you have to know what the parent did.  With this change it would allow you to add TAGs to be able to figure out what is going on.

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

No.

### How was this patch tested?

A new unit test is included in the PR.

Closes #29224 from andygrove/insert-aqe-rule.

Authored-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 11:03:57 -07:00
Kent Yao d3596c04b0 [SPARK-32406][SQL] Make RESET syntax support single configuration reset
### What changes were proposed in this pull request?

This PR extends the RESET command to support reset SQL configuration one by one.

### Why are the changes needed?

Currently, the reset command only supports restore all of the runtime configurations to their defaults. In most cases, users do not want this,  but just want to restore one or a small group of settings.
The SET command can work as a workaround for this, but you have to keep the defaults in your mind or by temp variables, which turns out not very convenient to use.

Hive supports this:
https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-BeelineExample

reset <key> | Resets the value of a particular configuration variable (key) to the default value.Note: If you misspell the variable name, Beeline will not show an error.
-- | --

PostgreSQL supports this too

https://www.postgresql.org/docs/9.1/sql-reset.html

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

yes, reset can restore one configuration now
### How was this patch tested?

add new unit tests.

Closes #29202 from yaooqinn/SPARK-32406.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 09:13:26 -07:00
Max Gekk 8bc799f920 [SPARK-32375][SQL] Basic functionality of table catalog v2 for JDBC
### What changes were proposed in this pull request?
This PR implements basic functionalities of the `TableCatalog` interface, so that end-users can use the JDBC as a catalog.

### Why are the changes needed?
To have at least one built implementation of Catalog Plugin API available to end users. JDBC is perfectly fit for this.

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

### How was this patch tested?
By new test suite `JDBCTableCatalogSuite`.

Closes #29168 from MaxGekk/jdbc-v2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-24 14:12:43 +00:00
Liang-Chi Hsieh 84efa04c57 [SPARK-32308][SQL] Move by-name resolution logic of unionByName from API code to analysis phase
### What changes were proposed in this pull request?

Currently the by-name resolution logic of `unionByName` is put in API code. This patch moves the logic to analysis phase.
See https://github.com/apache/spark/pull/28996#discussion_r453460284.

### Why are the changes needed?

Logically we should do resolution in analysis phase. This refactoring cleans up API method and makes consistent resolution.

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

No

### How was this patch tested?

Unit tests.

Closes #29107 from viirya/move-union-by-name.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-24 04:33:18 +00:00
Max Gekk 19e3ed765a [SPARK-32415][SQL][TESTS] Enable tests for JSON option: allowNonNumericNumbers
### What changes were proposed in this pull request?
Enable two tests from `JsonParsingOptionsSuite`:
- `allowNonNumericNumbers off`
- `allowNonNumericNumbers on`

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the enabled tests.

Closes #29207 from MaxGekk/allowNonNumericNumbers-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-24 09:55:36 +09:00
Max Gekk 658e87471c [SPARK-30648][SQL][FOLLOWUP] Refactoring of JsonFilters: move config checking out
### What changes were proposed in this pull request?
Refactoring of `JsonFilters`:
- Add an assert to the `skipRow` method to check the input `index`
- Move checking of the SQL config `spark.sql.json.filterPushdown.enabled` from `JsonFilters` to `JacksonParser`.

### Why are the changes needed?
1. The assert should catch incorrect usage of `JsonFilters`
2. The config checking out of `JsonFilters` makes it consistent with `OrderedFilters` (see https://github.com/apache/spark/pull/29145).
3. `JsonFilters` can be used by other datasource in the future and don't depend from the JSON configs.

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

### How was this patch tested?
By existing tests suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.execution.datasources.json.*"
$ build/sbt "test:testOnly org.apache.spark.sql.catalyst.json.*"
```

Closes #29206 from MaxGekk/json-filters-pushdown-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-24 09:54:11 +09:00
Sean Owen be2eca22e9 [SPARK-32398][TESTS][CORE][STREAMING][SQL][ML] Update to scalatest 3.2.0 for Scala 2.13.3+
### What changes were proposed in this pull request?

Updates to scalatest 3.2.0. Though it looks large, it is 99% changes to the new location of scalatest classes.

### Why are the changes needed?

3.2.0+ has a fix that is required for Scala 2.13.3+ compatibility.

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

No, only affects tests.

### How was this patch tested?

Existing tests.

Closes #29196 from srowen/SPARK-32398.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 16:20:17 -07:00
Terry Kim 35345e30e5 [SPARK-32374][SQL] Disallow setting properties when creating temporary views
### What changes were proposed in this pull request?

Currently, you can specify properties when creating a temporary view. However, the specified properties are not used and can be misleading.

This PR propose to disallow specifying properties when creating temporary views.

### Why are the changes needed?

To avoid confusion by disallowing specifying unused properties.

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

Yes, now if you create a temporary view with properties, the operation will fail:
```
scala> sql("CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1")
org.apache.spark.sql.catalyst.parser.ParseException:
Operation not allowed: CREATE TEMPORARY VIEW ... TBLPROPERTIES (property_name = property_value, ...)(line 1, pos 0)

== SQL ==
CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1
^^^

```

### How was this patch tested?

Added tests

Closes #29167 from imback82/disable_properties_temp_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:32:10 +00:00
yi.wu a8e3de36e7 [SPARK-32280][SPARK-32372][SQL] ResolveReferences.dedupRight should only rewrite attributes for ancestor nodes of the conflict plan
### What changes were proposed in this pull request?

This PR refactors `ResolveReferences.dedupRight` to make sure it only rewrite attributes for ancestor nodes of the conflict plan.

### Why are the changes needed?

This is a bug fix.

```scala
sql("SELECT name, avg(age) as avg_age FROM person GROUP BY name")
  .createOrReplaceTempView("person_a")
sql("SELECT p1.name, p2.avg_age FROM person p1 JOIN person_a p2 ON p1.name = p2.name")
  .createOrReplaceTempView("person_b")
sql("SELECT * FROM person_a UNION SELECT * FROM person_b")
  .createOrReplaceTempView("person_c")
sql("SELECT p1.name, p2.avg_age FROM person_c p1 JOIN person_c p2 ON p1.name = p2.name").show()
```
When executing the above query, we'll hit the error:

```scala
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: Resolved attribute(s) avg_age#231 missing from name#223,avg_age#218,id#232,age#234,name#233 in operator !Project [name#233, avg_age#231]. Attribute(s) with the same name appear in the operation: avg_age. Please check if the right attribute(s) are used.;;
...
```

The plan below is the problematic plan which is the right plan of a `Join` operator. And, it has conflict plans comparing to the left plan. In this problematic plan, the first `Aggregate` operator (the one under the first child of `Union`) becomes a conflict plan compares to the left one and has a rewrite attribute pair as  `avg_age#218` -> `avg_age#231`. With the current `dedupRight` logic, we'll first replace this `Aggregate` with a new one, and then rewrites the attribute `avg_age#218` from bottom to up. As you can see, projects with the attribute `avg_age#218` of the second child of the `Union` can also be replaced with `avg_age#231`(That means we also rewrite attributes for non-ancestor plans for the conflict plan). Ideally, the attribute `avg_age#218` in the second `Aggregate` operator (the one under the second child of `Union`) should also be replaced. But it didn't because it's an `Alias` while we only rewrite `Attribute` yet. Therefore, the project above the second `Aggregate` becomes unresolved.

```scala
:

:
+- SubqueryAlias p2
   +- SubqueryAlias person_c
      +- Distinct
         +- Union
            :- Project [name#233, avg_age#231]
            :  +- SubqueryAlias person_a
            :     +- Aggregate [name#233], [name#233, avg(cast(age#234 as bigint)) AS avg_age#231]
            :        +- SubqueryAlias person
            :           +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234]
            :              +- ExternalRDD [obj#165]
            +- Project [name#233 AS name#227, avg_age#231 AS avg_age#228]
               +- Project [name#233, avg_age#231]
                  +- SubqueryAlias person_b
                     +- !Project [name#233, avg_age#231]
                        +- Join Inner, (name#233 = name#223)
                           :- SubqueryAlias p1
                           :  +- SubqueryAlias person
                           :     +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234]
                           :        +- ExternalRDD [obj#165]
                           +- SubqueryAlias p2
                              +- SubqueryAlias person_a
                                 +- Aggregate [name#223], [name#223, avg(cast(age#224 as bigint)) AS avg_age#218]
                                    +- SubqueryAlias person
                                       +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#222, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#223, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#224]
                                          +- ExternalRDD [obj#165]
```

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

Yes, users would no longer hit the error after this fix.

### How was this patch tested?

Added test.

Closes #29166 from Ngone51/impr-dedup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:24:47 +00:00
Wenchen Fan aa54dcf193 [SPARK-32251][SQL][TESTS][FOLLOWUP] improve SQL keyword test
### What changes were proposed in this pull request?

Improve the `SQLKeywordSuite` so that:
1. it checks keywords under default mode as well
2. it checks if there are typos in the doc (found one and fixed in this PR)

### Why are the changes needed?

better test coverage

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

no

### How was this patch tested?

N/A

Closes #29200 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:02:38 +00:00
Dongjoon Hyun aed8dbab1d [SPARK-32364][SQL][FOLLOWUP] Add toMap to return originalMap and documentation
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/29160. We already removed the indeterministicity. This PR aims the following for the existing code base.
1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

### Why are the changes needed?

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`.

Closes #29191 from dongjoon-hyun/SPARK-32364-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 06:28:08 -07:00
LantaoJin 182566bf57 [SPARK-32237][SQL] Resolve hint in CTE
### What changes were proposed in this pull request?
This PR is to move `Substitution` rule before `Hints` rule in `Analyzer` to avoid hint in CTE not working.

### Why are the changes needed?
Below SQL in Spark3.0 will throw AnalysisException, but it works in Spark2.x
```sql
WITH cte AS (SELECT /*+ REPARTITION(3) */ T.id, T.data FROM $t1 T)
SELECT cte.id, cte.data FROM cte
```
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`cte.id`' given input columns: [cte.data, cte.id]; line 3 pos 7;
'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl

'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl
```

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

### How was this patch tested?
Add a unit test

Closes #29062 from LantaoJin/SPARK-32237.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:10:45 +00:00
Takuya UESHIN 46169823c0 [SPARK-30616][SQL][FOLLOW-UP] Use only config key name in the config doc
### What changes were proposed in this pull request?

This is a follow-up of #28852.

This PR to use only config name; otherwise the doc for the config entry shows the entire details of the referring configs.

### Why are the changes needed?

The doc for the newly introduced config entry shows the entire details of the referring configs.

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

The doc for the config entry will show only the referring config keys.

### How was this patch tested?

Existing tests.

Closes #29194 from ueshin/issues/SPARK-30616/fup.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:07:30 +00:00
Kent Yao b151194299 [SPARK-32392][SQL] Reduce duplicate error log for executing sql statement operation in thrift server
### What changes were proposed in this pull request?

This PR removes the duplicated error log which has been logged in `org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation#execute` but logged again in `runInternal`.

Besides, the log4j configuration for SparkExecuteStatementOperation is turned off because it's not very friendly for Jenkins

### Why are the changes needed?

remove the duplicated error log for better  user experience

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

Yes, less log in thrift server's driver log

### How was this patch tested?

locally verified the result in target/unit-test.log

Closes #29189 from yaooqinn/SPARK-32392.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-23 10:12:13 +09:00