Commit graph

8149 commits

Author SHA1 Message Date
Zhu, Lipeng 3b37bfde2a [SPARK-27949][SQL] Support SUBSTRING(str FROM n1 [FOR n2]) syntax
## What changes were proposed in this pull request?

Currently, function `substr/substring`'s usage is like `substring(string_expression, n1 [,n2])`.

But, the ANSI SQL defined the pattern for substr/substring is like `SUBSTRING(str FROM n1 [FOR n2])`. This gap makes some inconvenient when we switch to the SparkSQL.

- ANSI SQL-92: http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt

Below are the mainly DB engines to support the ANSI standard for substring.
- PostgreSQL https://www.postgresql.org/docs/9.1/functions-string.html
- MySQL https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_substring
- Redshift https://docs.aws.amazon.com/redshift/latest/dg/r_SUBSTRING.html
- Teradata https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/XnePye0Cwexw6Pny_qnxVA

**Oracle, SQL Server, Hive, Presto don't have this additional syntax.**

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24802 from lipzhu/SPARK-27949.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 09:05:10 -07:00
Chaerim Yeo c1bb3316bd [SPARK-27425][SQL] Add count_if function
## What changes were proposed in this pull request?

Add `count_if` function which returns the number of records satisfying a given condition.

There is no aggregation function like this in Spark, so we need to write like
- `COUNT(CASE WHEN some_condition THEN 1 END)` or
- `SUM(CASE WHEN some_condition THEN 1 END)`, 
which looks painful.

This kind of function is already supported in Presto, BigQuery and even Excel.
- Presto: [`count_if`](https://prestodb.github.io/docs/current/functions/aggregate.html#count_if)
- BigQuery: [`countif`](https://cloud.google.com/bigquery/docs/reference/standard-sql/aggregate_functions?hl=en#countif)
- Excel: [`COUNTIF`](https://support.office.com/en-us/article/countif-function-e0de10c6-f885-4e71-abb4-1f464816df34?omkt=en-US&ui=en-US&rs=en-US&ad=US) (It is a little different from above twos)

## How was this patch tested?

This patch is tested by unit test.

Closes #24335 from cryeo/SPARK-27425.

Authored-by: Chaerim Yeo <yeochaerim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-10 19:51:55 +09:00
sandeep katta 773cfde680 [SPARK-27917][SQL] canonical form of CaseWhen object is incorrect
## What changes were proposed in this pull request?

For caseWhen Object canonicalized is not handled

for e.g let's consider below CaseWhen Object
    val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
    val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))

caseWhenObj1.canonicalized **ouput** is as below

CASE WHEN ACCESS_CHECK#0 THEN A END (**Before Fix)**

**After Fix** : CASE WHEN none#0 THEN A END

So when there will be aliasref like below statements, semantic equals will fail. Sematic equals returns true if the canonicalized form of both the expressions are same.

val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
val aliasAttrRef = attrRef.withName("access_check")
val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))
val caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A"))))

**assert(caseWhenObj2.semanticEquals(caseWhenObj1.semanticEquals) fails**

**caseWhenObj1.canonicalized**

Before Fix:CASE WHEN ACCESS_CHECK#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END
**caseWhenObj2.canonicalized**

Before Fix:CASE WHEN access_check#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END

## How was this patch tested?
Added UT

Closes #24766 from sandeep-katta/caseWhenIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 00:33:47 -07:00
HyukjinKwon f984f6acfe Revert "[SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)"
## What changes were proposed in this pull request?

This PR reverts 9c4eb99c52 for the reasons below:

1. An alternative was not considered properly, https://github.com/apache/spark/pull/24734#issuecomment-500101639 https://github.com/apache/spark/pull/24734#issuecomment-500102340 https://github.com/apache/spark/pull/24734#issuecomment-499202982 - I opened a PR https://github.com/apache/spark/pull/24826

2. 9c4eb99c52 fixed timely flushing which behaviour is somewhat hacky and the timing isn't also guaranteed (in case each batch takes longer to process).

3. For pipelining for smaller batches, looks it's better to allow to configure buffer size rather than having another factor to flush

## How was this patch tested?

N/A

Closes #24827 from HyukjinKwon/revert-flush.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 08:28:31 -07:00
HyukjinKwon 6dcf09becc [SPARK-27971][SQL][R] MapPartitionsInRWithArrowExec.evaluate shouldn't eagerly read the first batch
## What changes were proposed in this pull request?

This PR is the same fix as https://github.com/apache/spark/pull/24816 but in vectorized `dapply` in SparkR.

## How was this patch tested?

Manually tested.

Closes #24818 from HyukjinKwon/SPARK-27971.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-09 11:40:20 +09:00
Gengliang Wang db0f6b4674 [SPARK-27961][SQL] DataSourceV2Relation should not have refresh method
## What changes were proposed in this pull request?

The newly added `Refresh` method in PR #24401 prevented the work of moving DataSourceV2Relation into catalyst. It calls `case table: FileTable => table.fileIndex.refresh()` while `FileTable` belongs to sql/core.

More importantly, Ryan Blue pointed out DataSourceV2Relation is immutable by design, it should not have refresh method.

## How was this patch tested?

Unit test

Closes #24815 from gengliangwang/removeRefreshTable.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 10:59:10 -07:00
Yuming Wang 2926890ffb [SPARK-27970][SQL] Support Hive 3.0 metastore
## What changes were proposed in this pull request?

It seems that some users are using Hive 3.0.0. This pr makes it support Hive 3.0 metastore.

## How was this patch tested?

unit tests

Closes #24688 from wangyum/SPARK-26145.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 15:24:07 -07:00
WeichenXu 9c4eb99c52 [SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)
## What changes were proposed in this pull request?

Flush batch timely for pandas UDF.

This could improve performance when multiple pandas UDF plans are pipelined.

When batch being flushed in time, downstream pandas UDFs will get pipelined as soon as possible, and pipeline will help hide the donwstream UDFs computation time. For example:

When the first UDF start computing on batch-3, the second pipelined UDF can start computing on batch-2, and the third pipelined UDF can start computing on batch-1.

If we do not flush each batch in time, the donwstream UDF's pipeline will lag behind too much, which may increase the total processing time.

I add flush at two places:
* JVM process feed data into python worker. In jvm side, when write one batch, flush it
* VM process read data from python worker output, In python worker side, when write one batch, flush it

If no flush, the default buffer size for them are both 65536. Especially in the ML case, in order to make realtime prediction, we will make batch size very small. The buffer size is too large for the case, which cause downstream pandas UDF pipeline lag behind too much.

### Note
* This is only applied to pandas scalar UDF.
* Do not flush for each batch. The minimum interval between two flush is 0.1 second. This avoid too frequent flushing when batch size is small. It works like:
```
        last_flush_time = time.time()
        for batch in iterator:
                writer.write_batch(batch)
                flush_time = time.time()
                if self.flush_timely and (flush_time - last_flush_time > 0.1):
                      stream.flush()
                      last_flush_time = flush_time
```

## How was this patch tested?

### Benchmark to make sure the flush do not cause performance regression
#### Test code:
```
numRows = ...
batchSize = ...

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', str(batchSize))
df = spark.range(1, numRows + 1, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 10

beg_time = time.time()
result = df.select(sum(fp1('a'))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```
#### Test Result:

 params        | Consume time (Before) | Consume time (After)
------------ | ----------------------- | ----------------------
numRows=100000000, batchSize=10000 | 23.43s | 24.64s
numRows=100000000, batchSize=1000 | 36.73s | 34.50s
numRows=10000000, batchSize=100 | 35.67s | 32.64s
numRows=1000000, batchSize=10 | 33.60s | 32.11s
numRows=100000, batchSize=1 | 33.36s | 31.82s

### Benchmark pipelined pandas UDF
#### Test code:
```
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))

```
#### Test Result:

**Before**: consume time: 63.57s
**After**: consume time: 32.43s
**So the PR improve performance by make downstream UDF get pipelined early.**

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #24734 from WeichenXu123/improve_pandas_udf_pipeline.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 14:02:43 -07:00
Liang-Chi Hsieh 527d936049 [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation
## What changes were proposed in this pull request?

When using `from_avro` to deserialize avro data to catalyst StructType format, if `ConvertToLocalRelation` is applied at the time, `from_avro` produces only the last value (overriding previous values).

The cause is `AvroDeserializer` reuses output row for StructType. Normally, it should be fine in Spark SQL. But `ConvertToLocalRelation` just uses `InterpretedProjection` to project local rows. `InterpretedProjection` creates new row for each output thro, it includes the same nested row object from `AvroDeserializer`. By the end, converted local relation has only last value.

I think there're two possible options:

1. Make `AvroDeserializer` output new row for StructType.
2. Use `InterpretedMutableProjection` in `ConvertToLocalRelation` and call `copy()` on output rows.

Option 2 is chose because previously `ConvertToLocalRelation` also creates new rows, this `InterpretedMutableProjection` + `copy()` shoudn't bring too much performance penalty. `ConvertToLocalRelation` should be arguably less critical, compared with `AvroDeserializer`.

## How was this patch tested?

Added test.

Closes #24805 from viirya/SPARK-27798.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-07 13:47:36 -07:00
Ryan Blue b30655bdef [SPARK-27965][SQL] Add extractors for v2 catalog transforms.
## What changes were proposed in this pull request?

Add extractors for v2 catalog transforms.

These extractors are used to match transforms that are equivalent to Spark's internal case classes. This makes it easier to work with v2 transforms.

## How was this patch tested?

Added test suite for the new extractors.

Closes #24812 from rdblue/SPARK-27965-add-transform-extractors.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 00:20:36 -07:00
liwensun eee3467b1e [SPARK-27938][SQL] Remove feature flag LEGACY_PASS_PARTITION_BY_AS_OPTIONS
## What changes were proposed in this pull request?
In PR https://github.com/apache/spark/pull/24365, we pass in the partitionBy columns as options in `DataFrameWriter`.  To make this change less intrusive for a patch release, we added a feature flag `LEGACY_PASS_PARTITION_BY_AS_OPTIONS` with the default to be false.

For 3.0, we should just do the correct behavior for DSV1, i.e., always passing partitionBy as options, and remove this legacy feature flag.

## How was this patch tested?
Existing tests.

Closes #24784 from liwensun/SPARK-27453-default.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-07 11:33:58 +09:00
Xiangrui Meng 4d770db0eb [SPARK-27968] ArrowEvalPythonExec.evaluate shouldn't eagerly read the first row
## What changes were proposed in this pull request?

Issued fixed in https://github.com/apache/spark/pull/24734 but that PR might takes longer to merge.

## How was this patch tested?

It should pass existing unit tests.

Closes #24816 from mengxr/SPARK-27968.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-06 15:45:44 -07:00
Yuming Wang eadb53824d [SPARK-27918][SQL] Port boolean.sql
## What changes were proposed in this pull request?

This PR is to port boolean.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/boolean.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/boolean.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
- [SPARK-27931](https://issues.apache.org/jira/browse/SPARK-27931): Accept 'on' and 'off' as input for boolean data type / Trim the string when cast to boolean type / Accept unique prefixes thereof
- [SPARK-27924](https://issues.apache.org/jira/browse/SPARK-27924): Support E061-14: Search Conditions

Also, found an inconsistent behavior:
- [SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Unsupported input throws an exception in PostgreSQL but Spark accepts it and sets the value to `NULL`, for example:
```sql
SELECT bool 'test' AS error; -- SELECT boolean('test') AS error;
```

## How was this patch tested?

N/A

Closes #24767 from wangyum/SPARK-27918.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 10:57:10 -07:00
Yuming Wang 4de96493ae [SPARK-27883][SQL] Port AGGREGATES.sql [Part 2]
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/aggregates.sql#L145-L350

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/aggregates.out#L499-L984

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:

- [SPARK-27877](https://issues.apache.org/jira/browse/SPARK-27877): Implement SQL-standard LATERAL subqueries
- [SPARK-27878](https://issues.apache.org/jira/browse/SPARK-27878): Support ARRAY(sub-SELECT) expressions
- [SPARK-27879](https://issues.apache.org/jira/browse/SPARK-27879): Implement bitwise integer aggregates(BIT_AND and BIT_OR)
- [SPARK-27880](https://issues.apache.org/jira/browse/SPARK-27880): Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY)

## How was this patch tested?

N/A

Closes #24743 from wangyum/SPARK-27883.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 09:28:59 -07:00
Ryan Blue d1371a2dad [SPARK-27964][SQL] Move v2 catalog update methods to CatalogV2Util
## What changes were proposed in this pull request?

Move methods that implement v2 catalog operations to CatalogV2Util so they can be used in #24768.

## How was this patch tested?

Behavior is validated by existing tests.

Closes #24813 from rdblue/SPARK-27964-add-catalog-v2-util.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 19:44:53 -07:00
Jordan Sanders 20e8843350 [MINOR][SQL] Skip warning if JOB_SUMMARY_LEVEL is set to NONE
## What changes were proposed in this pull request?

I believe the log message: `Committer $committerClass is not a ParquetOutputCommitter and cannot create job summaries. Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.` is at odds with the `if` statement that logs the warning. Despite the instructions in the warning, users still encounter the warning if `JOB_SUMMARY_LEVEL` is already set to `NONE`.

This pull request introduces a change to skip logging the warning if `JOB_SUMMARY_LEVEL` is set to `NONE`.

## How was this patch tested?

I built to make sure everything still compiled and I ran the existing test suite. I didn't feel it was worth the overhead to add a test to make sure a log message does not get logged, but if reviewers feel differently, I can add one.

Closes #24808 from jmsanders/master.

Authored-by: Jordan Sanders <jmsanders@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 14:57:36 -07:00
Ryan Blue 5d6758c0e7 [SPARK-27857][SQL] Move ALTER TABLE parsing into Catalyst
## What changes were proposed in this pull request?

This moves parsing logic for `ALTER TABLE` into Catalyst and adds parsed logical plans for alter table changes that use multi-part identifiers. This PR is similar to SPARK-27108, PR #24029, that created parsed logical plans for create and CTAS.

* Create parsed logical plans
* Move parsing logic into Catalyst's AstBuilder
* Convert to DataSource plans in DataSourceResolution
* Parse `ALTER TABLE ... SET LOCATION ...` separately from the partition variant
* Parse `ALTER TABLE ... ALTER COLUMN ... [TYPE dataType] [COMMENT comment]` [as discussed on the dev list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Syntax-for-table-DDL-td25197.html#a25270)
* Parse `ALTER TABLE ... RENAME COLUMN ... TO ...`
* Parse `ALTER TABLE ... DROP COLUMNS ...`

## How was this patch tested?

* Added new tests in Catalyst's `DDLParserSuite`
* Moved converted plan tests from SQL `DDLParserSuite` to `PlanResolutionSuite`
* Existing tests for regressions

Closes #24723 from rdblue/SPARK-27857-add-alter-table-statements-in-catalyst.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 13:21:30 -07:00
Jacek Laskowski 6c28ef144d [SPARK-27933][SS] Extracting common purge behaviour to the parent StreamExecution
Extracting the common purge "behaviour" to the parent StreamExecution.

## How was this patch tested?

No added behaviour so relying on existing tests.

Closes #24781 from jaceklaskowski/StreamExecution-purge.

Authored-by: Jacek Laskowski <jacek@japila.pl>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-05 12:39:31 -05:00
Wenchen Fan 8b6232b119 [SPARK-27521][SQL] Move data source v2 to catalyst module
## What changes were proposed in this pull request?

Currently we are in a strange status that, some data source v2 interfaces(catalog related) are in sql/catalyst, some data source v2 interfaces(Table, ScanBuilder, DataReader, etc.) are in sql/core.

I don't see a reason to keep data source v2 API in 2 modules. If we should pick one module, I think sql/catalyst is the one to go.

Catalyst module already has some user-facing stuff like DataType, Row, etc. And we have to update `Analyzer` and `SessionCatalog` to support the new catalog plugin, which needs to be in the catalyst module.

This PR can solve the problem we have in https://github.com/apache/spark/pull/24246

## How was this patch tested?

existing tests

Closes #24416 from cloud-fan/move.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 09:55:55 -07:00
Yuming Wang 3f102a8229 [SPARK-27749][SQL] hadoop-3.2 support hive-thriftserver
## What changes were proposed in this pull request?

This PR mainly makes the following changes to make `hadoop-3.2` support `sql/hive-thriftserver`:
1. Upgrade [`TCLIService.thrift`](https://github.com/apache/hive/blob/rel/release-2.3.5/service-rpc/if/TCLIService.thrift) and related code to Hive 2.3.5 because of [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442)(Note that we only migrate code without adding features, such as [HIVE-4924](https://issues.apache.org/jira/browse/HIVE-4924) and [HIVE-15473](https://issues.apache.org/jira/browse/HIVE-15473)).
2. Use slf4j as logging facade because of [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237).
3. Port [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) to compatible with Hive 2.3.

## How was this patch tested?

Exiting test

Closes #24628 from wangyum/SPARK-27749.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:40:05 -07:00
LantaoJin 18834e85d0 [SPARK-27899][SQL] Refactor getTableOption() to extract a common method
## What changes were proposed in this pull request?

This is a part of #24774, to reduce the code changes made by that.

## How was this patch tested?

Exist UTs.

Closes #24803 from LantaoJin/SPARK-27899_refactor.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:36:25 -07:00
Ryan Blue de73a54269 [SPARK-27909][SQL] Do not run analysis inside CTE substitution
## What changes were proposed in this pull request?

This updates CTE substitution to avoid needing to run all resolution rules on each substituted expression. Running resolution rules was previously used to avoid infinite recursion. In the updated rule, CTE plans are substituted as sub-queries from right to left. Using this scope-based order, it is not necessary to replace multiple CTEs at the same time using `resolveOperatorsDown`. Instead, `resolveOperatorsUp` is used to replace each CTE individually.

By resolving using `resolveOperatorsUp`, this no longer needs to run all analyzer rules on each substituted expression. Previously, this was done to apply `ResolveRelations`, which would throw an `AnalysisException` for all unresolved relations so that unresolved relations that may cause recursive substitutions were not left in the plan. Because this is no longer needed, `ResolveRelations` no longer needs to throw `AnalysisException` and resolution can be done in multiple rules.

## How was this patch tested?

Existing tests in `SQLQueryTestSuite`, `cte.sql`.

Closes #24763 from rdblue/SPARK-27909-fix-cte-substitution.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-04 14:46:13 -07:00
David Vogelbacher f9ca8ab196 [SPARK-27805][PYTHON] Propagate SparkExceptions during toPandas with arrow enabled
## What changes were proposed in this pull request?
Similar to https://github.com/apache/spark/pull/24070, we now propagate SparkExceptions that are encountered during the collect in the java process to the python process.

Fixes https://jira.apache.org/jira/browse/SPARK-27805

## How was this patch tested?
Added a new unit test

Closes #24677 from dvogelbacher/dv/betterErrorMsgWhenUsingArrow.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-04 10:10:27 -07:00
williamwong d5715a9b23 [SPARK-27772][SQL][TEST] Refactor SQLTestUtils to use tryWithSafeFinally
## What changes were proposed in this pull request?

The current `SQLTestUtils` created many `withXXX` utility functions to clean up tables/views/caches created for testing purpose. Java's `try-with-resources` statement does something similar, but it does not mask exception throwing in the try block with any exception caught in the 'close()' statement. Exception caught in the 'close()' statement would add as a suppressed exception instead.

This PR standardizes those 'withXXX' function to use`Utils.tryWithSafeFinally` function, which does something similar to Java's try-with-resources statement. The purpose of this proposal is to help developers to identify what actually breaks their tests.

## How was this patch tested?
Existing testcases.

Closes #24747 from William1104/feature/SPARK-27772-2.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-04 09:26:24 -05:00
Michael Chirico 3ddc26ddd8 [MINOR][DOCS] Add a clarifying note to str_to_map documentation
I was quite surprised by the following behavior:

`SELECT str_to_map('1:2|3:4', '|')`

vs

`SELECT str_to_map(replace('1:2|3:4', '|', ','))`

The documentation does not make clear at all what's going on here, but a [dive into the source code shows](fa0d4bf699/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala (L461-L466)) that `split` is being used and in turn the interpretation of `split`'s arguments as RegEx is clearly documented.

## What changes were proposed in this pull request?

Documentation clarification

## How was this patch tested?

N/A

Closes #23888 from MichaelChirico/patch-2.

Authored-by: Michael Chirico <michaelchirico4@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-04 16:58:25 +09:00
Gengliang Wang d1937c1479 [SPARK-27926][SQL] Allow altering table add columns with CSVFileFormat/JsonFileFormat provider
## What changes were proposed in this pull request?

In the previous work of csv/json migration, CSVFileFormat/JsonFileFormat is removed in the table provider whitelist of `AlterTableAddColumnsCommand.verifyAlterTableAddColumn`:
https://github.com/apache/spark/pull/24005
https://github.com/apache/spark/pull/24058

This is regression. If a table is created with Provider `org.apache.spark.sql.execution.datasources.csv.CSVFileFormat` or `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`, Spark should allow the "alter table add column" operation.

## How was this patch tested?

Unit test

Closes #24776 from gengliangwang/v1Table.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-03 23:51:05 -07:00
Dongjoon Hyun 8486680b34 [SPARK-24544][SQL][FOLLOWUP] Remove a wrong warning on Hive fallback lookup
## What changes were proposed in this pull request?

This PR is a follow-up of https://github.com/apache/spark/pull/21790 which causes a regression to show misleading warnings always at first invocation for all Hive function. Hive fallback lookup should not be warned. It's a normal process in function lookups.

**CURRENT (Showing `NoSuchFunctionException` and working)**
```scala
scala> sql("select histogram_numeric(a,2) from values(1) T(a)").show
19/06/02 22:02:10 WARN HiveSessionCatalog: Encountered a failure during looking up
function: org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException:
Undefined function: 'histogram_numeric'. This function is neither a registered temporary
function nor a permanent function registered in the database 'default'.;
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.failFunctionLookup(SessionCatalog.scala:1234)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1302)
...
+------------------------+
|histogram_numeric( a, 2)|
+------------------------+
|            [[1.0, 1.0]]|
+------------------------+
```

## How was this patch tested?

Manually execute the above query.

Closes #24773 from dongjoon-hyun/SPARK-24544.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-03 00:04:00 -07:00
HyukjinKwon 8b18ef5c7b [MINOR] Avoid hardcoded py4j-0.10.8.1-src.zip in Scala
## What changes were proposed in this pull request?

This PR targets to deduplicate hardcoded `py4j-0.10.8.1-src.zip` in order to make py4j upgrade easier.

## How was this patch tested?

N/A

Closes #24770 from HyukjinKwon/minor-py4j-dedup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:23:17 -07:00
Dongjoon Hyun 809821a283 [SPARK-27920][SQL][TEST] Add interceptParseException test utility function
## What changes were proposed in this pull request?

This PR aims to add `interceptParseException` test utility function to `AnalysisTest` to reduce the duplications of `intercept` functions.

## How was this patch tested?

Pass the Jenkins with the updated test suites.

Closes #24769 from dongjoon-hyun/SPARK-27920.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:11:35 -07:00
Yuming Wang d53b61c311 [SPARK-27831][SQL][TEST] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Please note that this pr need test with `maven` and `sbt`.

Closes #24751 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 20:23:08 -07:00
Liang-Chi Hsieh 2a88fffacb [SPARK-27873][SQL] columnNameOfCorruptRecord should not be checked with column names in CSV header when disabling enforceSchema
## What changes were proposed in this pull request?

If we want to keep corrupt record when reading CSV, we provide a new column into the schema, that is `columnNameOfCorruptRecord`. But this new column isn't actually a column in CSV header. So if `enforceSchema` is disabled, `CSVHeaderChecker` throws a exception complaining that number of column in CSV header isn't equal to that in the schema.

## How was this patch tested?

Added test.

Closes #24757 from viirya/SPARK-27873.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 11:09:26 +09:00
HyukjinKwon f5317f10b2 [SPARK-27893][SQL][PYTHON] Create an integrated test base for Python, Scalar Pandas, Scala UDF by sql files
## What changes were proposed in this pull request?

This PR targets to add an integrated test base for various UDF test cases so that Scalar UDF, Python UDF and Scalar Pandas UDFs can be tested in SBT & Maven tests.

### Problem

One of the problems we face is that: `ExtractPythonUDFs` (for Python UDF and Scalar Pandas UDF) has unevaluable expressions that always has to be wrapped with special plans. This special rule seems producing many issues, for instance, SPARK-27803, SPARK-26147, SPARK-26864, SPARK-26293, SPARK-25314 and SPARK-24721.

### Why do we have less test cases dedicated for SQL and plans with Python UDFs?

We have virtually no such SQL (or plan) dedicated tests in PySpark to catch such issues because:
  - A developer should know all the analyzer, the optimizer, SQL, PySpark, Py4J and version differences in Python to write such good test cases
  - To test plans, we should access to plans in JVM via Py4J which is tricky, messy and duplicates Scala test cases
  - Usually we just add end-to-end test cases in PySpark therefore there are not so many dedicated examples to refer to write in PySpark

It is also a non-trivial overhead to switch test base and method (IMHO).

### How does this PR fix?

This PR adds Python UDF and Scalar Pandas UDF into our `*.sql` file based test base in runtime of SBT / Maven test cases. It generates Python-pickled instance (consisting of return type and Python native function) that is used in Python or Scalar Pandas UDF and directly brings into JVM.

After that, (we don't interact via Py4J) run the tests directly in JVM - we can just register and run Python UDF and Scalar Pandas UDF in JVM.

Currently, I only integrated this change into SQL file based testing. This is how works with test files under `udf` directory:

After the test files under 'inputs/udf' directory are detected, it creates three test cases:
  - Scala UDF test case with a Scalar UDF registered named 'udf'.
  - Python UDF test case with a Python UDF registered named 'udf' iff Python executable and pyspark are available.
  - Scalar Pandas UDF test case with a Scalar Pandas UDF registered named 'udf' iff Python executable, pandas, pyspark and pyarrow are available.

Therefore, UDF test cases should have single input and output files but executed by three different types of UDFs.

For instance,

```sql
CREATE TEMPORARY VIEW ta AS
SELECT udf(a) AS a, udf('a') AS tag FROM t1
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t2;

CREATE TEMPORARY VIEW tb AS
SELECT udf(a) AS a, udf('a') AS tag FROM t3
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t4;

SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag;
```

will be ran 3 times with Scalar UDF, Python UDF and Scalar Pandas UDF each.

### Appendix

Plus, this PR adds `IntegratedUDFTestUtils` which enables to test and execute Python UDF and Scalar Pandas UDFs as below:

To register Python UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestPythonUDF(name = "udf"), spark)
```

To register Scalar Pandas UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestScalarPandasUDF(name = "udf"), spark)
```

 To use it in Scala API:

```scala
spark.select(expr("udf(1)").show()
```

 To use it in SQL:

```scala
sql("SELECT udf(1)").show()
```

This util could be used in the future for better coverage with Scala API combinations as well.

## How was this patch tested?

Tested via the command below:

```bash
build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-inner-join.sql"
```

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (5 seconds, 47 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (4 seconds, 335 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF (5 seconds, 423 milliseconds)
```

[python] unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 577 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [pyton] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [pyton]. !!! IGNORED !!!
```

pyspark unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 991 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [python] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

pandas and/or pyarrow unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 713 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (3 seconds, 89 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

Closes #24752 from HyukjinKwon/udf-tests.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 10:03:36 +09:00
HyukjinKwon db48da87f0 [SPARK-27834][SQL][R][PYTHON] Make separate PySpark/SparkR vectorization configurations
## What changes were proposed in this pull request?

`spark.sql.execution.arrow.enabled` was added when we add PySpark arrow optimization.
Later, in the current master, SparkR arrow optimization was added and it's controlled by the same configuration `spark.sql.execution.arrow.enabled`.

There look two issues about this:

1. `spark.sql.execution.arrow.enabled` in PySpark was added from 2.3.0 whereas SparkR optimization was added 3.0.0. The stability is different so it's problematic when we change the default value for one of both optimization first.

2. Suppose users want to share some JVM by PySpark and SparkR. They are currently forced to use the optimization for all or none if the configuration is set globally.

This PR proposes two separate configuration groups for PySpark and SparkR about Arrow optimization:

- Deprecate `spark.sql.execution.arrow.enabled`
- Add `spark.sql.execution.arrow.pyspark.enabled` (fallback to `spark.sql.execution.arrow.enabled`)
- Add `spark.sql.execution.arrow.sparkr.enabled`
- Deprecate `spark.sql.execution.arrow.fallback.enabled`
- Add `spark.sql.execution.arrow.pyspark.fallback.enabled ` (fallback to `spark.sql.execution.arrow.fallback.enabled`)

Note that `spark.sql.execution.arrow.maxRecordsPerBatch` is used within JVM side for both.
Note that `spark.sql.execution.arrow.fallback.enabled` was added due to behaviour change. We don't need it in SparkR - SparkR side has the automatic fallback.

## How was this patch tested?

Manually tested and some unittests were added.

Closes #24700 from HyukjinKwon/separate-sparkr-arrow.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 10:01:37 +09:00
Ajith 3806887afb [SPARK-27907][SQL] HiveUDAF should return NULL in case of 0 rows
## What changes were proposed in this pull request?

When query returns zero rows, the HiveUDAFFunction throws NPE

## CASE 1:
create table abc(a int)
select histogram_numeric(a,2) from abc // NPE
```
Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 0, localhost, executor driver): java.lang.NullPointerException
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:471)
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:315)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.eval(interfaces.scala:543)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:231)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:122)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

## CASE 2:
create table abc(a int)
insert into abc values (1)
select histogram_numeric(a,2) from abc where a=3 // NPE

```
Job aborted due to stage failure: Task 0 in stage 4.0 failed 1 times, most recent failure: Lost task 0.0 in stage 4.0 (TID 5, localhost, executor driver): java.lang.NullPointerException
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:477)
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:315)
at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.serializeAggregateBufferInPlace(interfaces.scala:570)
at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$6(AggregationIterator.scala:254)
at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:122)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```

Hence add a check not avoid NPE

## How was this patch tested?

Added new UT case

Closes #24762 from ajithme/hiveudaf.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 10:54:21 -07:00
Marco Gaido 93db7b870d [SPARK-27684][SQL] Avoid conversion overhead for primitive types
## What changes were proposed in this pull request?

As outlined in the JIRA by JoshRosen, our conversion mechanism from catalyst types to scala ones is pretty inefficient for primitive data types. Indeed, in these cases, most of the times we are adding useless calls to `identity` function or anyway to functions which return the same value. Using the information we have when we generate the code, we can avoid most of these overheads.

## How was this patch tested?

Here is a simple test which shows the benefit that this PR can bring:
```
test("SPARK-27684: perf evaluation") {
    val intLongUdf = ScalaUDF(
      (a: Int, b: Long) => a + b, LongType,
      Literal(1) :: Literal(1L) :: Nil,
      true :: true :: Nil,
      nullable = false)

    val plan = generateProject(
      MutableProjection.create(Alias(intLongUdf, s"udf")() :: Nil),
      intLongUdf)
    plan.initialize(0)

    var i = 0
    val N = 100000000
    val t0 = System.nanoTime()
    while(i < N) {
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      i += 1
    }
    val t1 = System.nanoTime()
    println(s"Avg time: ${(t1 - t0).toDouble / N} ns")
  }
```
The output before the patch is:
```
Avg time: 51.27083294 ns
```
after, we get:
```
Avg time: 11.85874227 ns
```
which is ~5X faster.

Moreover a benchmark has been added for Scala UDF. The output after the patch can be seen in this PR, before the patch, the output was:
```
================================================================================================
UDF with mixed input types
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to string:       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to string wholestage off            257            287          42          0,4        2569,5       1,0X
long/nullable int/string to string wholestage on            158            172          18          0,6        1579,0       1,6X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to option:       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to option wholestage off            104            107           5          1,0        1037,9       1,0X
long/nullable int/string to option wholestage on             80             92          12          1,2         804,0       1,3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to primitive:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to primitive wholestage off             71             76           7          1,4         712,1       1,0X
long/nullable int to primitive wholestage on             64             71           6          1,6         636,2       1,1X

================================================================================================
UDF with primitive types
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to string:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to string wholestage off             60             60           0          1,7         600,3       1,0X
long/nullable int to string wholestage on             55             64           8          1,8         551,2       1,1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to option:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to option wholestage off             66             73           9          1,5         663,0       1,0X
long/nullable int to option wholestage on             30             32           2          3,3         300,7       2,2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to primitive:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to primitive wholestage off             32             35           5          3,2         316,7       1,0X
long/nullable int/string to primitive wholestage on             41             68          17          2,4         414,0       0,8X
```
The improvements are particularly visible in the second case, ie. when only primitive types are used as inputs.

Closes #24636 from mgaido91/SPARK-27684.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-05-30 17:09:19 -07:00
Gengliang Wang 49e7387741 [SPARK-27849][SQL][FOLLOWUP][TEST-MAVEN] Fix the testing regex in DataSourceScanRedactionTest
## What changes were proposed in this pull request?

As explained in https://github.com/apache/spark/pull/24719#pullrequestreview-243064785, the regex `file:/[\\w-_/]+` contains possible characters I have met in the Jenkins tests.
However, we still miss the  `.` symbol:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/6415/testReport/junit/org.apache.spark.sql.execution/DataSourceV2ScanExecRedactionSuite/treeString_is_redacted/ :
```
orc *********(redacted).7/sql/core/target/tmp/spark-7ff5f81d-069a-4b5d-9d9a-808addeef115
```

This PR is to fix it by matching any character except `]` or spaces.
## How was this patch tested?

Unit test

Closes #24745 from gengliangwang/fixRegex.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-30 13:45:12 -07:00
Dongjoon Hyun 955eef95b3 Revert "[SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency"
This reverts commit 24180c00e0.
2019-05-30 10:06:55 -07:00
John Zhuge a44b00dfe0 [SPARK-27813][SQL] DataSourceV2: Add DropTable logical operation
## What changes were proposed in this pull request?

Support DROP TABLE from V2 catalogs.
Move DROP TABLE into catalyst.
Move parsing tests for DROP TABLE/VIEW to PlanResolutionSuite to validate existing behavior.
Add new tests fo catalyst parser suite.
Separate DROP VIEW into different code path from DROP TABLE.
Move DROP VIEW into catalyst as a new operator.
Add a meaningful exception to indicate view is not currently supported in v2 catalog.

## How was this patch tested?

New unit tests.
Existing unit tests in catalyst and sql core.

Closes #24686 from jzhuge/SPARK-27813-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-31 00:56:07 +08:00
Yuming Wang db3e746b64 [SPARK-27875][CORE][SQL][ML][K8S] Wrap all PrintWriter with Utils.tryWithResource
## What changes were proposed in this pull request?

This pr wrap all `PrintWriter` with `Utils.tryWithResource` to prevent resource leak.

## How was this patch tested?

Existing test

Closes #24739 from wangyum/SPARK-27875.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-30 19:54:32 +09:00
John Zhuge 953b8e8206 [SPARK-26946][SQL][FOLLOWUP] Require lookup function
## What changes were proposed in this pull request?

Require the lookup function with interface LookupCatalog. Rationale is in the review comments below.

Make `Analyzer` abstract. BaseSessionStateBuilder and HiveSessionStateBuilder implements lookupCatalog with a call to SparkSession.catalog().

Existing test cases and those that don't need catalog lookup will use a newly added `TestAnalyzer` with a default lookup function that throws` CatalogNotFoundException("No catalog lookup function")`.

Rewrote the unit test for LookupCatalog to demonstrate the interface can be used anywhere, not just Analyzer.

Removed Analyzer parameter `lookupCatalog` because we can override in the following manner:
```
new Analyzer() {
  override def lookupCatalog(name: String): CatalogPlugin = ???
}
```

## How was this patch tested?

Existing unit tests.

Closes #24689 from jzhuge/SPARK-26946-follow.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-30 09:22:42 +08:00
Gengliang Wang c1007c2f7c [SPARK-27849][SQL] Redact treeString of FileTable and DataSourceV2ScanExecBase
## What changes were proposed in this pull request?

To follow https://github.com/apache/spark/pull/17397, the output of FileTable and DataSourceV2ScanExecBase can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non-privileged users.

This PR is to add a redaction facility for these outputs to resolve the issue. A user can enable this by setting a regex in the same spark.redaction.string.regex configuration as V1.
## How was this patch tested?

Unit test

Closes #24719 from gengliangwang/RedactionSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 13:32:21 -07:00
Josh Rosen 19aaf0f784 [SPARK-27829][SQL] In Dataset.joinWith() inner joins, don't nest data before shuffling
## What changes were proposed in this pull request?

In order to support outer joins with null top-level objects, SPARK-15441 modified Dataset.joinWith to project both inputs into single-column structs prior to the join.

For inner joins, however, this step is unnecessary and actually harms performance: performing the nesting before the join increases the shuffled data size. As an optimization for inner joins only, we can move this nesting to occur after the join (effectively switching back to the pre-SPARK-15441 behavior; see #13425).

## How was this patch tested?

Existing tests, which I strengthened to also make assertions about the join result's nullability (since this guards against a bug I almost introduced during prototyping).

Here's a quick `spark-shell` experiment demonstrating the reduction in shuffle size:

```scala
// With --conf spark.shuffle.compress=false
sql("set spark.sql.autoBroadcastJoinThreshold=-1") // for easier shuffle measurements
case class Foo(a: Long, b: Long)
val left = spark.range(10000).map(x => Foo(x, x))
val right = spark.range(10000).map(x => Foo(x, x))
left.joinWith(right, left("a") === right("a"), "inner").rdd.count()
left.joinWith(right, left("a") === right("a"), "left").rdd.count()
```

With inner join (which benefits from this PR's optimization) we shuffle 546.9 KiB. With left outer join (whose plan hasn't changed, therefore being a representation of the state before this PR) we shuffle 859.4 KiB. Shuffle compression (which is enabled by default) narrows this gap a bit: with compression, outer joins shuffle about 12% more than inner joins.

Closes #24693 from JoshRosen/fast-join-with-for-inner-joins.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-29 16:12:24 +08:00
Yuming Wang 67582fdbfe [SPARK-27737][SQL][FOLLOW-UP] Move sql/hive-thriftserver/v2.3.4 to sql/hive-thriftserver/v2.3.5
## What changes were proposed in this pull request?

This pr moves `sql/hive-thriftserver/v2.3.4` to `sql/hive-thriftserver/v2.3.5` based on ([comment](https://github.com/apache/spark/pull/24628#issuecomment-496459258)).

## How was this patch tested?

N/A

Closes #24728 from wangyum/SPARK-27737-thriftserver.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 16:08:58 -07:00
Yuming Wang 4e61de4380 [SPARK-27863][SQL] Metadata files and temporary files should not be counted as data files
## What changes were proposed in this pull request?
[`DataSourceUtils.isDataPath(path)`](https://github.com/apache/spark/blob/v2.4.3/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala#L95) should be `DataSourceUtils.isDataPath(status.getPath)`.

This pr fix this issue.

## How was this patch tested?

unit tests

Closes #24725 from wangyum/SPARK-27863.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 09:28:35 -07:00
gengjiaan c30b5297bc [SPARK-27776][SQL] Avoid duplicate Java reflection in DataSource.
## What changes were proposed in this pull request?

I checked the code of
`org.apache.spark.sql.execution.datasources.DataSource`
, there exists duplicate Java reflection.
`sourceSchema`,`createSource`,`createSink`,`resolveRelation`,`writeAndRead`, all the methods call the `providingClass.getConstructor().newInstance()`.
The instance of `providingClass` is stateless, such as:
`KafkaSourceProvider`
`RateSourceProvider`
`TextSocketSourceProvider`
`JdbcRelationProvider`
`ConsoleSinkProvider`

AFAIK, Java reflection will result in significant performance issue.
The oracle website [https://docs.oracle.com/javase/tutorial/reflect/index.html](https://docs.oracle.com/javase/tutorial/reflect/index.html) contains some performance description about Java reflection:

```
Performance Overhead
Because reflection involves types that are dynamically resolved, certain Java virtual machine optimizations can not be performed. Consequently, reflective operations have slower performance than their non-reflective counterparts, and should be avoided in sections of code which are called frequently in performance-sensitive applications.
```

I have found some performance cost test of Java reflection as follows:
[https://blog.frankel.ch/performance-cost-of-reflection/](https://blog.frankel.ch/performance-cost-of-reflection/) contains performance cost test.
[https://stackoverflow.com/questions/435553/java-reflection-performance](https://stackoverflow.com/questions/435553/java-reflection-performance) has a discussion of java reflection.

So I think should avoid duplicate Java reflection and reuse the instance of `providingClass`.

## How was this patch tested?

Exists UT.

Closes #24647 from beliefer/optimize-DataSource.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-28 09:26:06 -05:00
wenxuanguan 35952cb42c [SPARK-27859][SS] Use efficient sorting instead of .sorted.reverse sequence
## What changes were proposed in this pull request?

descending sort in HDFSMetadataLog.getLatest instead of two action of ascending sort and reverse

## How was this patch tested?

Jenkins

Closes #24711 from wenxuanguan/bug-fix-hdfsmetadatalog.

Authored-by: wenxuanguan <choose_home@126.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-27 21:53:23 -07:00
Wenchen Fan 6506616b97 [SPARK-27803][SQL][PYTHON] Fix column pruning for Python UDF
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22104 , we create the python-eval nodes at the end of the optimization phase, which causes a problem.

After the main optimization batch, Filter and Project nodes are usually pushed to the bottom, near the scan node. However, if we extract Python UDFs from Filter/Project, and create a python-eval node under Filter/Project, it will break column pruning/filter pushdown of the scan node.

There are some hacks in the `ExtractPythonUDFs` rule, to duplicate the column pruning and filter pushdown logic. However, it has some bugs as demonstrated in the new test case(only column pruning is broken). This PR removes the hacks and re-apply the column pruning and filter pushdown rules explicitly.

**Before:**

```
...
== Analyzed Logical Plan ==
a: bigint
Project [a#168L]
+- Filter dummyUDF(a#168L)
   +- Relation[a#168L,b#169L] parquet

== Optimized Logical Plan ==
Project [a#168L]
+- Project [a#168L, b#169L]
   +- Filter pythonUDF0#174: boolean
      +- BatchEvalPython [dummyUDF(a#168L)], [a#168L, b#169L, pythonUDF0#174]
         +- Relation[a#168L,b#169L] parquet

== Physical Plan ==
*(2) Project [a#168L]
+- *(2) Project [a#168L, b#169L]
   +- *(2) Filter pythonUDF0#174: boolean
      +- BatchEvalPython [dummyUDF(a#168L)], [a#168L, b#169L, pythonUDF0#174]
         +- *(1) FileScan parquet [a#168L,b#169L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/_1/bzcp960d0hlb988k90654z2w0000gp/T/spark-798bae3c-a2..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
```

**After:**

```
...
== Analyzed Logical Plan ==
a: bigint
Project [a#168L]
+- Filter dummyUDF(a#168L)
   +- Relation[a#168L,b#169L] parquet

== Optimized Logical Plan ==
Project [a#168L]
+- Filter pythonUDF0#174: boolean
   +- BatchEvalPython [dummyUDF(a#168L)], [pythonUDF0#174]
      +- Project [a#168L]
         +- Relation[a#168L,b#169L] parquet

== Physical Plan ==
*(2) Project [a#168L]
+- *(2) Filter pythonUDF0#174: boolean
   +- BatchEvalPython [dummyUDF(a#168L)], [pythonUDF0#174]
      +- *(1) FileScan parquet [a#168L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/_1/bzcp960d0hlb988k90654z2w0000gp/T/spark-9500cafb-78..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint>
```

## How was this patch tested?

new test

Closes #24675 from cloud-fan/python.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-27 21:39:59 +09:00
Dilip Biswal 5060647bb1 [SPARK-27782][SQL] Use '#' to mark expression id embedded in the name field of SubqueryExec operator
## What changes were proposed in this pull request?
This is a minor pr to use `#` as a marker for expression id that is embedded in the name field of SubqueryExec operator.

## How was this patch tested?
Added a small test in SubquerySuite.

Closes #24652 from dilipbiswal/subquery-name.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 20:47:25 -07:00
Yuming Wang 447bfdec83 [SPARK-27844][SQL] Avoid hard-coded config: spark.rdd.parallelListingThreshold in SQL module
## What changes were proposed in this pull request?

Avoid hard-coded config: `spark.rdd.parallelListingThreshold`.

## How was this patch tested?

N/A

Closes #24708 from wangyum/spark.rdd.parallelListingThreshold.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 09:00:01 -07:00
Yuming Wang 193304b51b [SPARK-27441][SQL][TEST] Add read/write tests to Hive serde tables
## What changes were proposed in this pull request?

The versions between Hive, Parquet and ORC after the built-in Hive upgraded to 2.3.5 for Hadoop 3.2:

- built-in Hive is 1.2.1.spark2:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | Hive built-in | 1.6.0
Apache Hive 1.2.1 | Hive built-in | 1.6.0

- built-in Hive is 2.3.5:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | 1.5.5 | [1.10.1](https://github.com/apache/spark/pull/24346)
Apache Hive 2.3.5 | 1.3.4 | 1.8.1

We should add a test for Hive Serde table. This pr adds tests to test read/write of all supported data types using Parquet and ORC.

## How was this patch tested?

unit tests

Closes #24345 from wangyum/SPARK-27441.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:35:58 -07:00
Yuming Wang dcacfc5da6 [SPARK-27074][SQL][test-hadoop3.2][test-maven] Hive 3.1 metastore support HiveClientImpl.runHive
## What changes were proposed in this pull request?

Hive 3.1.1's `CommandProcessor` have 2 changes:
1. [HIVE-17626](https://issues.apache.org/jira/browse/HIVE-17626)(Hive 3.0.0) add ReExecDriver. So the current code path is: 02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L736-L742)
We can disable `hive.query.reexecution.enabled` to workaround this change.
2. [HIVE-18238](http://issues.apache.org/jira/browse/HIVE-18238)(Hive 3.0.0) changed the `Driver.close()` function return type. We can workaround it by ` driver.getClass.getMethod("close").invoke(driver)`

So Hive 3.1 metastore could support `HiveClientImpl.runHive` after this pr.

## How was this patch tested?

unit tests

Closes #23992 from wangyum/SPARK-27074.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:24:41 -07:00
rrusso2007 ebd1431a5a [SPARK-27801][SQL] Improve performance of InMemoryFileIndex.listLeafFiles for HDFS directories with many files
## What changes were proposed in this pull request?

InMemoryFileIndex.listLeafFiles should use listLocatedStatus for DistributedFileSystem. DistributedFileSystem overrides the listLocatedStatus method in order to do it with 1 single namenode call thus saving thousands of calls to getBlockLocations.

Currently in InMemoryFileIndex, all directory listings are done using FileSystem.listStatus following by individual calls to FileSystem.getFileBlockLocations. This is painstakingly slow for folders that have large numbers of files because this process happens serially and parallelism is only applied at the folder level, not the file level.

FileSystem also provides another API listLocatedStatus which returns the LocatedFileStatus objects that already have the block locations. In FileSystem main class this just delegates to listStatus and getFileBlockLocations similarly to the way Spark does it. However when HDFS specifically is the backing file system, DistributedFileSystem overrides this method and simply makes one single call to the namenode to retrieve the directory listing with the block locations. This avoids potentially thousands or more calls to namenode and also is more consistent because files will either exist with locations or not exist instead of having the FileNotFoundException exception case.

For our example directory with 6500 files, the load time of spark.read.parquet was reduced 96x from 76 seconds to .8 seconds. This savings only goes up with the number of files in the directory.

In the pull request instead of using this method always which could lead to a FileNotFoundException that could be tough to decipher in the default FileSystem implementation, this method is only used when the FileSystem is a DistributedFileSystem and otherwise the old logic still applies.

## How was this patch tested?

test suite ran

Closes #24672 from rrusso2007/master.

Authored-by: rrusso2007 <rrusso2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-25 15:49:30 -07:00
Yesheng Ma 5e3520f7f4 [SPARK-27809][SQL] Make optional clauses order insensitive for CREATE DATABASE/VIEW SQL statement
## What changes were proposed in this pull request?

Each time, when I write a complex CREATE DATABASE/VIEW statements, I have to open the .g4 file to find the EXACT order of clauses in CREATE TABLE statement. When the order is not right, I will get A strange confusing error message generated from ANTLR4.

The original g4 grammar for CREATE VIEW is
```
CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [db_name.]view_name
  [(col_name1 [COMMENT col_comment1], ...)]
  [COMMENT table_comment]
  [TBLPROPERTIES (key1=val1, key2=val2, ...)]
AS select_statement
```
The proposal is to make the following clauses order insensitive.
```
  [COMMENT table_comment]
  [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```
–
The original g4 grammar for CREATE DATABASE is
```
CREATE (DATABASE|SCHEMA) [IF NOT EXISTS] db_name
  [COMMENT comment_text]
  [LOCATION path]
  [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]
```
The proposal is to make the following clauses order insensitive.
```
  [COMMENT comment_text]
  [LOCATION path]
  [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]
```
## How was this patch tested?

By adding new unit tests to test duplicate clauses and modifying some existing unit tests to test whether those clauses are actually order insensitive

Closes #24681 from yeshengm/create-view-parser.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 15:19:14 -07:00
maryannxue de13f70ce1 [SPARK-27824][SQL] Make rule EliminateResolvedHint idempotent
## What changes were proposed in this pull request?

This fix prevents the rule EliminateResolvedHint from being applied again if it's already applied.

## How was this patch tested?

Added new UT.

Closes #24692 from maryannxue/eliminatehint-bug.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 11:25:22 -07:00
Wenchen Fan 7d318bfe90 [SPARK-26356][SQL] remove SaveMode from data source v2
## What changes were proposed in this pull request?

In data source v1, save mode specified in `DataFrameWriter` is passed to data source implementation directly, and each data source can define its own behavior about save mode. This is confusing and we want to get rid of save mode in data source v2.

For data source v2, we expect data source to implement the `TableCatalog` API, and end-users use SQL(or the new write API described in [this doc](https://docs.google.com/document/d/1gYm5Ji2Mge3QBdOliFV5gSPTKlX4q1DCBXIkiyMv62A/edit?ts=5ace0718#heading=h.e9v1af12g5zo)) to acess data sources. The SQL API has very clear semantic and we don't need save mode at all.

However, for simple data sources that do not have table management (like a JIRA data source, a noop sink, etc.), it's not ideal to ask them to implement the `TableCatalog` API, and throw exception here and there.

`TableProvider` API is created for simple data sources. It can only get tables, without any other table management methods. This means, it can only deal with existing tables.

`TableProvider` fits well with `DataStreamReader` and `DataStreamWriter`, as they can only read/write existing tables. However, `TableProvider` doesn't fit `DataFrameWriter` well, as the save mode requires more than just get table. More specifically, `ErrorIfExists` mode needs to check if table exists, and create table. `Ignore` mode needs to check if table exists. When end-users specify `ErrorIfExists` or `Ignore` mode and write data to `TableProvider` via `DataFrameWriter`, Spark fails the query and asks users to use `Append` or `Overwrite` mode.

The file source is in the middle of `TableProvider` and `TableCatalog`: it's simple but it can check table(path) exists and create table(path). That said, file source supports all the save modes.

Currently file source implements `TableProvider`, and it's not working because `TableProvider` doesn't support `ErrorIfExists` and `Ignore` modes. Ideally we should create a new API for path-based data sources, but to unblock the work of file source v2 migration, this PR proposes to special-case file source v2 in `DataFrameWriter`, to make it work.

This PR also removes `SaveMode` from data source v2, as now only the internal file source v2 needs it.

## How was this patch tested?

existing tests

Closes #24233 from cloud-fan/file.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 10:45:46 -07:00
Yuming Wang 24180c00e0 [SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Closes #24695 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-24 10:33:34 -07:00
Ryan Blue 6b28497d6f [SPARK-27732][SQL] Add v2 CreateTable implementation.
## What changes were proposed in this pull request?

This adds a v2 implementation of create table:
* `CreateV2Table` is the logical plan, named using v2 to avoid conflicting with the existing plan
* `CreateTableExec` is the physical plan

## How was this patch tested?

Added resolution and v2 SQL tests.

Closes #24617 from rdblue/SPARK-27732-add-v2-create-table.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-24 11:13:22 +08:00
gatorsmile f94247ec90 [SPARK-27770][SQL][PART 1] Port AGGREGATES.sql
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. 02ddd49932/src/test/regress/sql/aggregates.sql (L1-L143)

The expected results can be found in the link: https://github.com/postgres/postgres/blob/master/src/test/regress/expected/aggregates.out

When porting the test cases, found three PostgreSQL specific features that do not exist in Spark SQL.
- https://issues.apache.org/jira/browse/SPARK-27765: Type Casts: expression::type
- https://issues.apache.org/jira/browse/SPARK-27766: Data type: POINT(x, y)
- https://issues.apache.org/jira/browse/SPARK-27767: Built-in function: generate_series

Also, found two bugs:
- https://issues.apache.org/jira/browse/SPARK-27768: Infinity, -Infinity, NaN should be recognized in a case insensitive manner
- https://issues.apache.org/jira/browse/SPARK-27769: Handling of sublinks within outer-level aggregates.

This PR also fixes the error message when the column can't be resolved.

For running the regression tests, this PR also added three tables `aggtest`, `onek` and `tenk1` from the postgreSQL data sets: 02ddd49932/src/test/regress/data

## How was this patch tested?
N/A

Closes #24640 from gatorsmile/addTestCase.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-23 16:34:37 -07:00
HyukjinKwon c1e555711b Revert "Revert "[SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values""
This reverts commit 855399bbad.
2019-05-24 05:36:17 +09:00
HyukjinKwon 1ba4011a7f Revert "Revert "[SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…""
This reverts commit 516b0fb537.
2019-05-24 05:36:08 +09:00
Wenchen Fan 1a68fc38f0 [SPARK-27816][SQL] make TreeNode tag type safe
## What changes were proposed in this pull request?

Add type parameter to `TreeNodeTag`.

## How was this patch tested?

existing tests

Closes #24687 from cloud-fan/tag.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-23 11:53:21 -07:00
HyukjinKwon 516b0fb537 Revert "[SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…"
This reverts commit 40668c53ed.
2019-05-24 03:17:06 +09:00
HyukjinKwon 855399bbad Revert "[SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values"
This reverts commit 42cb4a2ccd.
2019-05-24 03:16:24 +09:00
Wenchen Fan a590a935b1 [SPARK-27806][SQL] byName/byPosition should apply to struct fields as well
## What changes were proposed in this pull request?

When writing a query to data source v2, we have 2 modes to resolve the input query's output: byName or byPosition.

For byName mode, we would reorder the top level columns according to the name, and add type cast if possible. If the names don't match, we fail.

For byPosition mode, we don't do the reorder, and just add type cast directly if possible.

However, for struct type fields, we always apply byName mode. We should ignore the name difference if byPosition mode is used.

## How was this patch tested?

new tests

Closes #24678 from cloud-fan/write.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-23 10:37:45 -07:00
Yuming Wang 76988dd4a2 [SPARK-27737][FOLLOW-UP][SQL][test-hadoop3.2] Update Hive test jars from 2.3.4 to 2.3.5
## What changes were proposed in this pull request?

This pr update `hive-contrib-2.3.4.jar` to `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.4.jar` to `hive-hcatalog-core-2.3.5.jar`.

## How was this patch tested?

Existing test

Closes #24673 from wangyum/SPARK-27737-hive.jar.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 08:29:06 -07:00
Dongjoon Hyun a24cdc00bf [SPARK-27800][SQL][HOTFIX][FOLLOWUP] Fix wrong answer on BitwiseXor test cases
This PR is a follow up of https://github.com/apache/spark/pull/24669 to fix the wrong answers used in test cases.

Closes #24674 from dongjoon-hyun/SPARK-27800.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 03:11:29 -07:00
Liu Xiao bf617996aa [SPARK-27800][SQL][DOC] Fix wrong answer of example for BitwiseXor
## What changes were proposed in this pull request?

Fix example for bitwise xor function. 3 ^ 5 should be 6 rather than 2.
- See https://spark.apache.org/docs/latest/api/sql/index.html#_14

## How was this patch tested?

manual tests

Closes #24669 from alex-lx/master.

Authored-by: Liu Xiao <hhdxlx@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 21:52:19 -07:00
David Vogelbacher 034cb139a1 [SPARK-27778][PYTHON] Fix toPandas conversion of empty DataFrame with Arrow enabled
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/22275 introduced a performance improvement where we send partitions out of order to python and then, as a last step, send the partition order as well.
However, if there are no partitions we will never send the partition order and we will get an "EofError" on the python side.
This PR fixes this by also sending the partition order if there are no partitions present.

## How was this patch tested?
New unit test added.

Closes #24650 from dvogelbacher/dv/fixNoPartitionArrowConversion.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 13:21:26 +09:00
Wenchen Fan 03c9e8adee [SPARK-24586][SQL] Upcast should not allow casting from string to other types
## What changes were proposed in this pull request?

When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet.

However, the current upcast behavior is a little weird, we don't allow up casting from string to numeric, but allow non-numeric types as the target, like boolean, date, etc.

As a result, `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution.

The motivation of the up cast is to prevent things like runtime NPE, it's more reasonable to make up cast stricter.

This PR does 2 things:
1. rename `Cast.canSafeCast` to `Cast.canUpcast`, and support complex typres
2. remove `Cast.mayTruncate` and replace it with `!Cast.canUpcast`

Note that, the up cast change also affects persistent view resolution. But since we don't support changing column types of an existing table, there is no behavior change here.

## How was this patch tested?

new tests

Closes #21586 from cloud-fan/cast.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-22 11:35:51 +08:00
Gengliang Wang c3c443ca8c [SPARK-27698][SQL] Add new method convertibleFilters for getting pushed down filters in Parquet file reader
## What changes were proposed in this pull request?

To return accurate pushed filters in Parquet file scan(https://github.com/apache/spark/pull/24327#pullrequestreview-234775673), we can process the original data source filters in the following way:
1. For "And" operators, split the conjunctive predicates and try converting each of them. After that
1.1 if partially predicate pushed down is allowed, return convertible results;
1.2 otherwise, return the whole predicate if convertible, or empty result if not convertible.

2. For "Or" operators, if both children can be  pushed down, it is partially or totally convertible; otherwise, return empty result

3. For other operators, they are not able to be partially pushed down.
2.1 if the entire predicate is convertible, return itself
2.2 otherwise, return an empty result.

This PR also contains code refactoring. Currently `ParquetFilters. createFilter ` accepts parameter `schema: MessageType` and create field mapping for every input filter. We can make it a class member and avoid creating the `nameToParquetField` mapping for every input filter.

## How was this patch tested?

Unit test

Closes #24597 from gengliangwang/refactorParquetFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-22 11:27:25 +08:00
Yuming Wang 6cd1efd0ae [SPARK-27737][SQL] Upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile
## What changes were proposed in this pull request?

This PR aims to upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile.

Release Notes - Hive - Version 2.3.5

- [[HIVE-21536](https://issues.apache.org/jira/browse/HIVE-21536)] - Backport HIVE-17764 to branch-2.3
- [[HIVE-21585](https://issues.apache.org/jira/browse/HIVE-21585)] - Upgrade branch-2.3 to ORC 1.3.4
- [[HIVE-21639](https://issues.apache.org/jira/browse/HIVE-21639)] - Spark test failed since HIVE-10632
- [[HIVE-21680](https://issues.apache.org/jira/browse/HIVE-21680)] - Backport HIVE-17644 to branch-2 and branch-2.3

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345394&styleName=Text&projectId=12310843

## How was this patch tested?

This PR is tested in two ways.
- Pass the Jenkins with the default configuration for `Hive Metastore Client` testing.
- Pass the Jenkins with `test-hadoop3.2` configuration for `Hadoop 3.2` testing.

Closes #24620 from wangyum/SPARK-27737.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 10:24:17 +09:00
williamwong 8442d94fb1 [SPARK-27248][SQL] refreshTable should recreate cache with same cache name and storage level
If we refresh a cached table, the table cache will be first uncached and then recache (lazily). Currently, the logic is embedded in CatalogImpl.refreshTable method.
The current implementation does not preserve the cache name and storage level. As a result, cache name and cache level could be changed after a REFERSH. IMHO, it is not what a user would expect.
I would like to fix this behavior by first save the cache name and storage level for recaching the table.

Two unit tests are added to make sure cache name is unchanged upon table refresh. Before applying this patch, the test created for qualified case would fail.

Closes #24221 from William1104/feature/SPARK-27248.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 11:37:16 -07:00
Liang-Chi Hsieh c033a3e1e6 [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans
## What changes were proposed in this pull request?

Because a temporary view is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain(true)
```

Before:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#2L]
+- SubqueryAlias `tmp001`
   +- Project [id#2L]
      +- SubqueryAlias `test2`
         +- Range (0, 5, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 5, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#0L]
+- SubqueryAlias `tmp001`
   +- Project [id#0L]
      +- SubqueryAlias `test`
         +- Range (0, 10, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 10, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)
```

Previous PR to this issue has a regression when to explain an explain statement, like `sql("explain select 1").explain(true)`. This new fix is following up with hvanhovell's advice at https://github.com/apache/spark/pull/24464#issuecomment-494165538.

Explain an explain:
```scala
scala> sql("explain select 1").explain(true)
== Parsed Logical Plan ==
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Analyzed Logical Plan ==
plan: string
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Optimized Logical Plan ==
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Physical Plan ==
Execute ExplainCommand
   +- ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false
```

Btw, I found there is a regression after applying hvanhovell's advice:

```scala
spark.readStream
      .format("org.apache.spark.sql.streaming.test")
      .load()
      .explain(true)
```

```scala
== Parsed Logical Plan ==
StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),None
), dummySource, [a#559]

== Analyzed Logical Plan ==
a: int
StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),Non$
), dummySource, [a#559]

== Optimized Logical Plan ==
org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
dummySource
== Physical Plan ==
org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
dummySource
```

So I did a change to that to fix it too.

## How was this patch tested?

Added test and manually test.

Closes #24654 from viirya/SPARK-27439-3.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-21 11:27:05 -07:00
Wenchen Fan 1e0facb60d [SQL][DOC][MINOR] update documents for Table and WriteBuilder
## What changes were proposed in this pull request?

Update the docs to reflect the changes made by https://github.com/apache/spark/pull/24129

## How was this patch tested?

N/A

Closes #24658 from cloud-fan/comment.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 09:29:06 -07:00
Josh Rosen 604aa1b045 [SPARK-27786][SQL] Fix Sha1, Md5, and Base64 codegen when commons-codec is shaded
## What changes were proposed in this pull request?

When running a custom build of Spark which shades `commons-codec`, the `Sha1` expression generates code which fails to compile:

```
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, Column 93: A method named "sha1Hex" is not declared in any enclosing class nor any supertype, nor through a static import
```

This is caused by an interaction between Spark's code generator and the shading: the current codegen template includes the string `org.apache.commons.codec.digest.DigestUtils.sha1Hex` as part of a larger string literal, preventing JarJarLinks from being able to replace the class name with the shaded class's name. As a result, the generated code still references the original unshaded class name name, triggering an error in case the original unshaded dependency isn't on the path.

This problem impacts the `Sha1`, `Md5`, and `Base64` expressions.

To fix this problem and allow for proper shading, this PR updates the codegen templates to replace the hardcoded class names with `${classof[<name>].getName}` calls.

## How was this patch tested?

Existing tests.

To ensure that I found all occurrences of this problem, I used IntelliJ's "Find in Path" to search for lines matching the regex `^(?!import|package).*(org|com|net|io)\.(?!apache\.spark)` and then filtered matches to inspect only non-test "Usage in string constants" cases. This isn't _perfect_ but I think it'll catch most cases.

Closes #24655 from JoshRosen/fix-shaded-apache-commons.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-21 21:18:34 +08:00
Dongjoon Hyun 039db879f4 Revert "[SPARK-27439][SQL] Explainging Dataset should show correct resolved plans"
This reverts commit 4b725e50a7.
2019-05-20 15:07:00 -07:00
Wenchen Fan 0e6601acdf [SPARK-27747][SQL] add a logical plan link in the physical plan
## What changes were proposed in this pull request?

It's pretty useful if we can convert a physical plan back to a logical plan, e.g., in https://github.com/apache/spark/pull/24389

This PR introduces a new feature to `TreeNode`, which allows `TreeNode` to carry some extra information via a mutable map, and keep the information when it's copied.

The planner leverages this feature to put the logical plan into the physical plan.

## How was this patch tested?

a test suite that runs all TPCDS queries and checks that some common physical plans contain the corresponding logical plans.

Closes #24626 from cloud-fan/link.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Peng Bo <bo.peng1019@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-20 13:42:25 -07:00
Yuming Wang 5dda1fe296 [SPARK-27699][FOLLOW-UP][SQL][test-hadoop3.2][test-maven] Fix hadoop-3.2 test error
## What changes were proposed in this pull request?

This pr fix `hadoop-3.2` test error:
```
- SPARK-27699 Converting disjunctions into ORC SearchArguments *** FAILED ***
  Expected "...SS_THAN_EQUALS a 10)[
  leaf-1 = (LESS_THAN a 1)
  ]expr = (or (not leaf...", but got "...SS_THAN_EQUALS a 10)[, leaf-1 = (LESS_THAN a 1), ]expr = (or (not leaf..." (HiveOrcFilterSuite.scala:445)
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105514/consoleFull

## How was this patch tested?

N/A

Closes #24639 from wangyum/SPARK-27699.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-20 13:04:05 -07:00
Yuming Wang 974b879220 [SPARK-27694][SQL] Support auto-updating table statistics for data source CTAS command
## What changes were proposed in this pull request?

This pr makes it support collect statistics when CTAS(create a data source table using the result of a query).

## How was this patch tested?

unit tests and manual tests:
```sql
bin/spark-sql --conf spark.sql.statistics.size.autoUpdate.enabled=true -S

spark-sql> CREATE TABLE spark_27694 USING parquet AS SELECT 'a', 'b';
spark-sql> DESC FORMATTED spark_27694;
a	string	NULL
b	string	NULL

# Detailed Table Information
Database	default
Table	spark_27694
Owner	root
Created Time	Mon May 13 19:45:33 GMT-07:00 2019
Last Access	Wed Dec 31 17:00:00 GMT-07:00 1969
Created By	Spark 3.0.0-SNAPSHOT
Type	MANAGED
Provider	parquet
Statistics	561 bytes
Location	file:/user/hive/warehouse/spark_27694
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
```

Closes #24596 from wangyum/SPARK-27694.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 22:29:40 -07:00
Ryan Blue bc46feaced [SPARK-27693][SQL] Add default catalog property
Add a SQL config property for the default v2 catalog.

Existing tests for regressions.

Closes #24594 from rdblue/SPARK-27693-add-default-catalog-config.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 21:30:52 -07:00
HyukjinKwon 2431ab0999 [SPARK-27771][SQL] Add SQL description for grouping functions (cube, rollup, grouping and grouping_id)
## What changes were proposed in this pull request?

Both look added as of 2.0 (see SPARK-12541 and SPARK-12706). I referred existing docs and examples in other API docs.

## How was this patch tested?

Manually built the documentation and, by running examples, by running `DESCRIBE FUNCTION EXTENDED`.

Closes #24642 from HyukjinKwon/SPARK-27771.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 19:26:20 -07:00
liuxian 9bca99b29b [SPARK-27552][SQL] The configuration hive.exec.stagingdir is invalid on Windows OS
## What changes were proposed in this pull requesst?
If we set `hive.exec.stagingdir=.test-staging\tmp`,
But the staging directory is still `.hive-staging` on Windows OS.

Reasons for failure:
Test code:
```
 val path = new Path("C:\\test\\hivetable")
  println("path.toString: " + path.toString)
  println("path.toUri.getPath: " + path.toUri.getPath)
```

Output:
```
path.toString: C:/test/hivetable
path.toUri.getPath: /C:/test/hivetable
```
 We can see that `path.toUri.getPath` has one more separator than `path.toString`,  and the separator is   ' / ',  not  ' \ '
So `stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")` will return false
## How was this patch tested?
1. Existed tests
2. Manual testing on Windows OS

Closes #24446 from 10110346/stagingdir.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-17 14:00:17 -05:00
Gengliang Wang e39e97b73a [SPARK-27699][SQL] Partially push down disjunctive predicated in Parquet/ORC
## What changes were proposed in this pull request?

Currently, in `ParquetFilters` and `OrcFilters`, if the child predicate of `Or` operator can't be entirely pushed down, the predicates will be thrown away.
In fact, the conjunctive predicates under `Or` operators can be partially pushed down.
For example, says `a` and `b` are convertible, while `c` can't be pushed down, the predicate
`a or (b and c)`
can be converted as
`(a or b) and (a or c)`
We can still push down `(a or b)`.
We can't push down disjunctive predicates only when one of its children is not partially convertible.

This PR also improve the filter pushing down logic in `DataSourceV2Strategy`. With partial filter push down in `Or` operator, the result of `pushedFilters()` might not exist in the mapping `translatedFilterToExpr`.  To fix it, this PR changes the mapping `translatedFilterToExpr` as leaf filter expression to `sources.filter`, and later on rebuild the whole expression with the mapping.
## How was this patch tested?

Unit test

Closes #24598 from gengliangwang/pushdownDisjunctivePredicates.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-17 19:25:24 +08:00
Wenchen Fan fc5bd6da77 [SPARK-27576][SQL] table capability to skip the output column resolution
## What changes were proposed in this pull request?

Currently we have an analyzer rule, which resolves the output columns of data source v2 writing plans, to make sure the schema of input query is compatible with the table.

However, not all data sources need this check. For example, the `NoopDataSource` doesn't care about the schema of input query at all.

This PR introduces a new table capability: ACCEPT_ANY_SCHEMA. If a table reports this capability, we skip resolving output columns for it during write.

Note that, we already skip resolving output columns for `NoopDataSource` because it implements `SupportsSaveMode`. However, `SupportsSaveMode` is a hack and will be removed soon.

## How was this patch tested?

new test cases

Closes #24469 from cloud-fan/schema-check.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 16:24:53 -07:00
Shixiong Zhu 6a317c8f01 [SPARK-27735][SS] Parsing interval string should be case-insensitive in SS
## What changes were proposed in this pull request?

Some APIs in Structured Streaming requires the user to specify an interval. Right now these APIs don't accept upper-case strings.

This PR adds a new method `fromCaseInsensitiveString` to `CalendarInterval` to support paring upper-case strings, and fixes all APIs that need to parse an interval string.

## How was this patch tested?

The new unit test.

Closes #24619 from zsxwing/SPARK-27735.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 13:58:27 -07:00
shivusondur c6a45e6f67 [SPARK-27722][SQL] removed the unsed "UnsafeKeyValueSorter" file.
## What changes were proposed in this pull request?

removed the unused "UnsafeKeyValueSorter.java" file

## How was this patch tested?

Ran Compilation and UT locally.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24622 from shivusondur/jira27722.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-16 18:22:06 +08:00
Wenchen Fan 3e30a98810 [SPARK-27674][SQL] the hint should not be dropped after cache lookup
## What changes were proposed in this pull request?

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

#20365 fixed this problem when the hint node is a root node. This PR fixes this problem for all the cases.

## How was this patch tested?

a new test

Closes #24580 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 15:47:52 -07:00
Yuming Wang 02c33694c8 [SPARK-27354][SQL] Move incompatible code from the hive-thriftserver module to sql/hive-thriftserver/v1.2.1
## What changes were proposed in this pull request?

When we upgraded the built-in Hive to 2.3.4, the current `hive-thriftserver` module is not compatible, such as these Hive changes:
1. [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442) HiveServer2: Refactor/repackage HiveServer2's Thrift code so that it can be used in the tasks
2. [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237) Use slf4j as logging facade
3. [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) HiveServer2: Support delegation token based connection when using http transport

So this PR moves the incompatible code to `sql/hive-thriftserver/v1.2.1` and copies it to `sql/hive-thriftserver/v2.3.4` for the next code review.

## How was this patch tested?

manual tests:
```
diff -urNa sql/hive-thriftserver/v1.2.1 sql/hive-thriftserver/v2.3.4
```

Closes #24282 from wangyum/SPARK-27354.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 14:52:08 -07:00
Xingbo Jiang 0bba5cf568 [SPARK-20774][SPARK-27036][SQL] Cancel the running broadcast execution on BroadcastTimeout
## What changes were proposed in this pull request?

In the existing code, a broadcast execution timeout for the Future only causes a query failure, but the job running with the broadcast and the computation in the Future are not canceled. This wastes resources and slows down the other jobs. This PR tries to cancel both the running job and the running hashed relation construction thread.

## How was this patch tested?

Add new test suite `BroadcastExchangeExec`

Closes #24595 from jiangxb1987/SPARK-20774.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 14:47:15 -07:00
Sean Owen bfb3ffe9b3 [SPARK-27682][CORE][GRAPHX][MLLIB] Replace use of collections and methods that will be removed in Scala 2.13 with work-alikes
## What changes were proposed in this pull request?

This replaces use of collection classes like `MutableList` and `ArrayStack` with workalikes that are available in 2.12, as they will be removed in 2.13. It also removes use of `.to[Collection]` as its uses was superfluous anyway. Removing `collection.breakOut` will have to wait until 2.13

## How was this patch tested?

Existing tests

Closes #24586 from srowen/SPARK-27682.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-15 09:29:12 -05:00
xy_xin fd9acf23b0 [SPARK-27713][SQL] Move org.apache.spark.sql.execution.* in catalyst to core
## What changes were proposed in this pull request?

`RecordBinaryComparator`, `UnsafeExternalRowSorter` and `UnsafeKeyValueSorter` now locates in catalyst, which should be moved to core, as they're used only in physical plan.

## How was this patch tested?

exist tests.

Closes #24607 from xianyinxin/SPARK-27713.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-15 15:24:21 +08:00
Ryan Blue 2da5b21834 [SPARK-24923][SQL] Implement v2 CreateTableAsSelect
## What changes were proposed in this pull request?

This adds a v2 implementation for CTAS queries

* Update the SQL parser to parse CREATE queries using multi-part identifiers
* Update `CheckAnalysis` to validate partitioning references with the CTAS query schema
* Add `CreateTableAsSelect` v2 logical plan and `CreateTableAsSelectExec` v2 physical plan
* Update create conversion from `CreateTableAsSelectStatement` to support the new v2 logical plan
* Update `DataSourceV2Strategy` to convert v2 CTAS logical plan to the new physical plan
* Add `findNestedField` to `StructType` to support reference validation

## How was this patch tested?

We have been running these changes in production for several months. Also:

* Add a test suite `CreateTablePartitioningValidationSuite` for new analysis checks
* Add a test suite for v2 SQL, `DataSourceV2SQLSuite`
* Update catalyst `DDLParserSuite` to use multi-part identifiers (`Seq[String]`)
* Add test cases to `PlanResolutionSuite` for v2 CTAS: known catalog and v2 source implementation

Closes #24570 from rdblue/SPARK-24923-add-v2-ctas.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-15 11:24:03 +08:00
Yuming Wang fee695d0cf [SPARK-27690][SQL] Remove materialized views first in HiveClientImpl.reset
## What changes were proposed in this pull request?

We should remove materialized view first otherwise(note that Hive 3.1 could reproduce this issue):
```scala
Cause: org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: DELETE on table 'TBLS' caused a violation of foreign key constraint 'MV_TABLES_USED_FK2' for key (4).  The statement has been rolled back.
at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedConnection.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.ConnectionChild.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeStatement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedPreparedStatement.executeBatchElement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeLargeBatch(Unknown Source)
```

## How was this patch tested?

Existing test

Closes #24592 from wangyum/SPARK-27690.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-14 09:05:22 -07:00
Sean Owen a10608cb82 [SPARK-27680][CORE][SQL][GRAPHX] Remove usage of Traversable
## What changes were proposed in this pull request?

This removes usage of `Traversable`, which is removed in Scala 2.13. This is mostly an internal change, except for the change in the `SparkConf.setAll` method. See additional comments below.

## How was this patch tested?

Existing tests.

Closes #24584 from srowen/SPARK-27680.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-14 09:14:56 -05:00
mingbo.pb 66f5a42ca5 [SPARK-27638][SQL] Cast string to date/timestamp in binary comparisons with dates/timestamps
## What changes were proposed in this pull request?

The below example works with both Mysql and Hive, however not with spark.

```
mysql> select * from date_test where date_col >= '2000-1-1';
+------------+
| date_col   |
+------------+
| 2000-01-01 |
+------------+
```
The reason is that Spark casts both sides to String type during date and string comparison for partial date support. Please find more details in https://issues.apache.org/jira/browse/SPARK-8420.

Based on some tests, the behavior of Date and String comparison in Hive and Mysql:
Hive: Cast to Date, partial date is not supported
Mysql: Cast to Date, certain "partial date" is supported by defining certain date string parse rules. Check out str_to_datetime in https://github.com/mysql/mysql-server/blob/5.5/sql-common/my_time.c

As below date patterns have been supported, the PR is to cast string to date when comparing string and date:
```
`yyyy`
`yyyy-[m]m`
`yyyy-[m]m-[d]d`
`yyyy-[m]m-[d]d `
`yyyy-[m]m-[d]d *`
`yyyy-[m]m-[d]dT*
```

## How was this patch tested?
UT has been added

Closes #24567 from pengbo/SPARK-27638.

Authored-by: mingbo.pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-14 17:10:36 +08:00
Liang-Chi Hsieh 8b0bdaa8e0 [SPARK-27671][SQL] Fix error when casting from a nested null in a struct
## What changes were proposed in this pull request?

When a null in a nested field in struct, casting from the struct throws error, currently.

```scala
scala> sql("select cast(struct(1, null) as struct<a:int,b:int>)").show
scala.MatchError: NullType (of class org.apache.spark.sql.types.NullType$)
  at org.apache.spark.sql.catalyst.expressions.Cast.castToInt(Cast.scala:447)
  at org.apache.spark.sql.catalyst.expressions.Cast.cast(Cast.scala:635)
  at org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castStruct$1(Cast.scala:603)
```

Similarly, inline table, which casts null in nested field under the hood, also throws an error.

```scala
scala> sql("select * FROM VALUES (('a', (10, null))), (('b', (10, 50))), (('c', null)) AS tab(x, y)").show
org.apache.spark.sql.AnalysisException: failed to evaluate expression named_struct('col1', 10, 'col2', NULL): NullType (of class org.apache.spark.sql.t
ypes.NullType$); line 1 pos 14
  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:47)
  at org.apache.spark.sql.catalyst.analysis.ResolveInlineTables.$anonfun$convert$6(ResolveInlineTables.scala:106)
```

This fixes the issue.

## How was this patch tested?

Added tests.

Closes #24576 from viirya/cast-null.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-13 12:40:46 -07:00
Yuming Wang f3ddd6f9da [SPARK-27402][SQL][TEST-HADOOP3.2][TEST-MAVEN] Fix hadoop-3.2 test issue(except the hive-thriftserver module)
## What changes were proposed in this pull request?

This pr fix hadoop-3.2 test issues(except the `hive-thriftserver` module):
1. Add `hive.metastore.schema.verification` and `datanucleus.schema.autoCreateAll` to HiveConf.
2. hadoop-3.2 support access the Hive metastore from 0.12 to 2.2

After [SPARK-27176](https://issues.apache.org/jira/browse/SPARK-27176) and this PR, we upgraded the built-in Hive to 2.3 when enabling the Hadoop 3.2+ profile. This upgrade fixes the following issues:
- [HIVE-6727](https://issues.apache.org/jira/browse/HIVE-6727): Table level stats for external tables are set incorrectly.
- [HIVE-15653](https://issues.apache.org/jira/browse/HIVE-15653): Some ALTER TABLE commands drop table stats.
- [SPARK-12014](https://issues.apache.org/jira/browse/SPARK-12014): Spark SQL query containing semicolon is broken in Beeline.
- [SPARK-25193](https://issues.apache.org/jira/browse/SPARK-25193): insert overwrite doesn't throw exception when drop old data fails.
- [SPARK-25919](https://issues.apache.org/jira/browse/SPARK-25919): Date value corrupts when tables are "ParquetHiveSerDe" formatted and target table is Partitioned.
- [SPARK-26332](https://issues.apache.org/jira/browse/SPARK-26332): Spark sql write orc table on viewFS throws exception.
- [SPARK-26437](https://issues.apache.org/jira/browse/SPARK-26437): Decimal data becomes bigint to query, unable to query.

## How was this patch tested?
This pr test Spark’s Hadoop 3.2 profile on jenkins and #24591 test Spark’s Hadoop 2.7 profile on jenkins

This PR close #24591

Closes #24391 from wangyum/SPARK-27402.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-13 10:35:26 -07:00
Liang-Chi Hsieh d169b0aac3 [SPARK-27653][SQL] Add max_by() and min_by() SQL aggregate functions
## What changes were proposed in this pull request?

This PR goes to add `max_by()` and `min_by()` SQL aggregate functions.

Quoting from the [Presto docs](https://prestodb.github.io/docs/current/functions/aggregate.html#max_by)

> max_by(x, y) → [same as x]
> Returns the value of x associated with the maximum value of y over all input values.

`min_by()` works similarly.

## How was this patch tested?

Added tests.

Closes #24557 from viirya/SPARK-27653.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-13 22:37:34 +08:00
zhoukang 126310ca68 [SPARK-26601][SQL] Make broadcast-exchange thread pool configurable
## What changes were proposed in this pull request?

Currently,thread number of broadcast-exchange thread pool is fixed and keepAliveSeconds is also fixed as 60s.

```
object BroadcastExchangeExec {
  private[execution] val executionContext = ExecutionContext.fromExecutorService(
    ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128))
}

 /**
   * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names
   * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer.
   */
  def newDaemonCachedThreadPool(
      prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = {
    val threadFactory = namedThreadFactory(prefix)
    val threadPool = new ThreadPoolExecutor(
      maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks
      maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used
      keepAliveSeconds,
      TimeUnit.SECONDS,
      new LinkedBlockingQueue[Runnable],
      threadFactory)
    threadPool.allowCoreThreadTimeOut(true)
    threadPool
  }
```

But some times, if the Thead object do not GC quickly it may caused server(driver) OOM. In such case,we need to make this thread pool configurable.
A case has described in https://issues.apache.org/jira/browse/SPARK-26601

## How was this patch tested?
UT

Closes #23670 from caneGuy/zhoukang/make-broadcat-config.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-13 20:40:21 +09:00
Gengliang Wang be6d39c379 [SPARK-27668][SQL] File source V2: support reporting statistics
## What changes were proposed in this pull request?

In File source V1, the statistics of `HadoopFsRelation` is `compressionFactor * sizeInBytesOfAllFiles`.
To follow it, we can implement the interface SupportsReportStatistics in FileScan and report the same statistics.

## How was this patch tested?

Unit test

Closes #24571 from gengliangwang/stats.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-13 14:16:11 +08:00
Wenchen Fan 9ff77b198e [SPARK-27675][SQL] do not use MutableColumnarRow in ColumnarBatch
## What changes were proposed in this pull request?

To move DS v2 API to the catalyst module, we can't refer to an internal class (`MutableColumnarRow`) in `ColumnarBatch`.

This PR creates a read-only version of `MutableColumnarRow`, and use it in `ColumnarBatch`.

close https://github.com/apache/spark/pull/24546

## How was this patch tested?

existing tests

Closes #24581 from cloud-fan/mutable-row.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-12 19:59:56 +09:00
HyukjinKwon c71f217de1 [SPARK-27673][SQL] Add since info to random, regex, null expressions
## What changes were proposed in this pull request?

We should add since info to all expressions.

SPARK-7886 Rand / Randn
af3746ce0d RLike, Like (I manually checked that it exists from 1.0.0)
SPARK-8262 Split
SPARK-8256 RegExpReplace
SPARK-8255 RegExpExtract
9aadcffabd Coalesce / IsNull / IsNotNull (I manually checked that it exists from 1.0.0)
SPARK-14541 IfNull / NullIf / Nvl / Nvl2
SPARK-9080 IsNaN
SPARK-9168 NaNvl

## How was this patch tested?

N/A

Closes #24579 from HyukjinKwon/SPARK-27673.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:24:04 -07:00
HyukjinKwon 3442fcaa9b [SPARK-27672][SQL] Add since info to string expressions
## What changes were proposed in this pull request?

This PR adds since information to the all string expressions below:

SPARK-8241 ConcatWs
SPARK-16276 Elt
SPARK-1995 Upper / Lower
SPARK-20750 StringReplace
SPARK-8266 StringTranslate
SPARK-8244 FindInSet
SPARK-8253 StringTrimLeft
SPARK-8260 StringTrimRight
SPARK-8267 StringTrim
SPARK-8247 StringInstr
SPARK-8264 SubstringIndex
SPARK-8249 StringLocate
SPARK-8252 StringLPad
SPARK-8259 StringRPad
SPARK-16281 ParseUrl
SPARK-9154 FormatString
SPARK-8269 Initcap
SPARK-8257 StringRepeat
SPARK-8261 StringSpace
SPARK-8263 Substring
SPARK-21007 Right
SPARK-21007 Left
SPARK-8248 Length
SPARK-20749 BitLength
SPARK-20749 OctetLength
SPARK-8270 Levenshtein
SPARK-8271 SoundEx
SPARK-8238 Ascii
SPARK-20748 Chr
SPARK-8239 Base64
SPARK-8268 UnBase64
SPARK-8242 Decode
SPARK-8243 Encode
SPARK-8245 format_number
SPARK-16285 Sentences

## How was this patch tested?

N/A

Closes #24578 from HyukjinKwon/SPARK-27672.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:11:12 -07:00
Eric Liang 80de449f51 [MINOR][TEST] Fix schema mismatch error
## What changes were proposed in this pull request?

- the accumulator warning is too verbose
- when a test fails with schema mismatch, you never see the error message / exception

Closes #24549 from ericl/test-nits.

Lead-authored-by: Eric Liang <ekl@databricks.com>
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-10 23:08:02 +08:00
Marco Gaido 78748b5752 [SPARK-27625][SQL] ScalaReflection support for annotated types
## What changes were proposed in this pull request?

If a type is annotated, `ScalaReflection` can fail if the datatype is an `Option`, a `Seq`, a `Map` and other similar types. This is because it assumes we are dealing with `TypeRef`, while types with annotations are `AnnotatedType`.

The PR deals with the case the annotation is present.

## How was this patch tested?

added UT

Closes #24564 from mgaido91/SPARK-27625.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-10 22:48:36 +08:00
Yuming Wang b5ffec12eb [SPARK-27563][FOLLOWUP] Fix to download new release from dist.apache.org
## What changes were proposed in this pull request?

`https://archive.apache.org/dist/spark/` does not have latest Spark 2.4.3:
<img src="https://user-images.githubusercontent.com/5399861/57288553-4264b600-70ec-11e9-8dcc-71b7589f5ad0.png" width="400">

This pr add `https://dist.apache.org/repos/dist/release/spark/` to mirrors list to download latest Spark.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105202/testReport/org.apache.spark.sql.hive/HiveExternalCatalogVersionsSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/

## How was this patch tested?

manual tests:
```
build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"  -Phive
```

Closes #24544 from wangyum/Unable-to-download-Spark-2.4.3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-09 08:45:52 -05:00
pgandhi 0969d7aa0c [SPARK-27207][SQL] : Ensure aggregate buffers are initialized again for So…
…rtBasedAggregate

Normally, the aggregate operations that are invoked for an aggregation buffer for User Defined Aggregate Functions(UDAF) follow the order like initialize(), update(), eval() OR initialize(), merge(), eval(). However, after a certain threshold configurable by spark.sql.objectHashAggregate.sortBased.fallbackThreshold is reached, ObjectHashAggregate falls back to SortBasedAggregator which invokes the merge or update operation without calling initialize() on the aggregate buffer.

## What changes were proposed in this pull request?

The fix here is to initialize aggregate buffers again when fallback to SortBasedAggregate operator happens.

## How was this patch tested?

The patch was tested as part of [SPARK-24935](https://issues.apache.org/jira/browse/SPARK-24935) as documented in PR https://github.com/apache/spark/pull/23778.

Closes #24149 from pgandhi999/SPARK-27207.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-09 11:12:20 +08:00
Gengliang Wang 78a403fab9 [SPARK-27627][SQL] Make option "pathGlobFilter" as a general option for all file sources
## What changes were proposed in this pull request?

### Background:
The data source option `pathGlobFilter` is introduced for Binary file format: https://github.com/apache/spark/pull/24354 , which can be used for filtering file names, e.g. reading `.png` files only while there is `.json` files in the same directory.

### Proposal:
Make the option `pathGlobFilter` as a general option for all file sources. The path filtering should happen in the path globbing on Driver.

### Motivation:
Filtering the file path names in file scan tasks on executors is kind of ugly.

### Impact:
1. The splitting of file partitions will be more balanced.
2. The metrics of file scan will be more accurate.
3. Users can use the option for reading other file sources.

## How was this patch tested?

Unit tests

Closes #24518 from gengliangwang/globFilter.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-09 08:41:43 +09:00
gengjiaan 57450ed9b7 [MINOR][SS] Rename secondLatestBatchId to secondLatestOffsets
## What changes were proposed in this pull request?

The mothod `populateStartOffsets` exists a inappropriate identifier `secondLatestBatchId`.
I think `secondLatestBatchId = latestBatchId - 1` and `offsetLog.get(latestBatchId - 1)` is a offset.
So I change the identifier as follows:
`secondLatestOffsets = offsetLog.get(latestBatchId - 1)`

## How was this patch tested?

Exists UT.

Closes #24550 from beliefer/fix-inappropriate-identifier.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-08 11:01:26 -07:00
Wenchen Fan bae5baae52 [SPARK-27642][SS] make v1 offset extends v2 offset
## What changes were proposed in this pull request?

To move DS v2 to the catalyst module, we can't make v2 offset rely on v1 offset, as v1 offset is in sql/core.

## How was this patch tested?

existing tests

Closes #24538 from cloud-fan/offset.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-07 23:03:15 -07:00
Yuming Wang 3ea44e52e7 [SPARK-27639][SQL] InMemoryTableScan shows the table name on UI if possible
## What changes were proposed in this pull request?
<img src="https://user-images.githubusercontent.com/5399861/57213799-7bccf100-701a-11e9-9872-d90b4a185dc6.png" width="200">

It only shows `InMemoryTableScan` when scanning InMemoryTable.
When there are many InMemoryTables, it is difficult to distinguish which one is what we are looking for. This PR show the table name when scanning InMemoryTable.

## How was this patch tested?

unit tests and manual tests

After this PR:
<img src="https://user-images.githubusercontent.com/5399861/57269120-d3219e80-70b8-11e9-9e56-1b5d4c071660.png" width="200">

Closes #24534 from wangyum/SPARK-27639.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 21:00:13 -07:00
Jose Torres 83f628b57d [SPARK-27253][SQL][FOLLOW-UP] Add a legacy flag to restore old session init behavior
## What changes were proposed in this pull request?

Add a legacy flag to restore the old session init behavior, where SparkConf defaults take precedence over configs in a parent session.

Closes #24540 from jose-torres/oss.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 20:04:09 -07:00
Ryan Blue 303ee3fce0 [SPARK-24252][SQL] Add TableCatalog API
## What changes were proposed in this pull request?

This adds the TableCatalog API proposed in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d).

For `TableCatalog` to use `Table`, it needed to be moved into the catalyst module where the v2 catalog API is located. This also required moving `TableCapability`. Most of the files touched by this PR are import changes needed by this move.

## How was this patch tested?

This adds a test implementation and contract tests.

Closes #24246 from rdblue/SPARK-24252-add-table-catalog-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-08 10:31:06 +08:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
Wenchen Fan 6ef45301a4 [SPARK-27579][SQL] remove BaseStreamingSource and BaseStreamingSink
## What changes were proposed in this pull request?

`BaseStreamingSource` and `BaseStreamingSink` is used to unify v1 and v2 streaming data source API in some code paths.

This PR removes these 2 interfaces, and let the v1 API extend v2 API to keep API compatibility.

The motivation is https://github.com/apache/spark/pull/24416 . We want to move data source v2 to catalyst module, but `BaseStreamingSource` and `BaseStreamingSink` are in sql/core.

## How was this patch tested?

existing tests

Closes #24471 from cloud-fan/streaming.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-06 20:41:57 +08:00
Liang-Chi Hsieh 4b725e50a7 [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans
## What changes were proposed in this pull request?

Because a review is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain(true)
```

Before:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#2L]
+- SubqueryAlias `tmp001`
   +- Project [id#2L]
      +- SubqueryAlias `test2`
         +- Range (0, 5, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 5, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#0L]
+- SubqueryAlias `tmp001`
   +- Project [id#0L]
      +- SubqueryAlias `test`
         +- Range (0, 10, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 10, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)
```

To fix it, this passes query execution of Dataset when explaining it. The query execution contains pre-analyzed plan which is consistent with Dataset's result.

## How was this patch tested?

Manually test and unit test.

Closes #24464 from viirya/SPARK-27439-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-05 23:19:19 -07:00
Dilip Biswal 6001d476ce [SPARK-27596][SQL] The JDBC 'query' option doesn't work for Oracle database
## What changes were proposed in this pull request?
**Description from JIRA**
For the JDBC option `query`, we use the identifier name to start with underscore: s"(${subquery}) _SPARK_GEN_JDBC_SUBQUERY_NAME${curId.getAndIncrement()}". This is not supported by Oracle.
The Oracle doesn't seem to support identifier name to start with non-alphabet character (unless it is quoted) and has length restrictions as well. [link](https://docs.oracle.com/cd/B19306_01/server.102/b14200/sql_elements008.htm)

In this PR, the generated alias name 'SPARK_GEN_JDBC_SUBQUERY_NAME<int value>' is fixed to remove "_" prefix and also the alias name is shortened to not exceed the identifier length limit.

## How was this patch tested?
Tests are added for MySql, Postgress, Oracle and DB2 to ensure enough coverage.

Closes #24532 from dilipbiswal/SPARK-27596.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-05 21:52:23 -07:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
Seth Fitzsimmons 5182aa25f0 [MINOR][DOCS] Correct date_trunc docs
## What changes were proposed in this pull request?

`date_trunc` argument order was flipped, phrasing was awkward.

## How was this patch tested?

Documentation-only.

Closes #24522 from mojodna/patch-2.

Authored-by: Seth Fitzsimmons <seth@mojodna.net>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 09:13:23 +09:00
sandeep katta c66ec43945 [SPARK-27555][SQL] HiveSerDe should fall back to hadoopconf if hive.default.fileformat is not found in SQLConf
## What changes were proposed in this pull request?

SQLConf does not load hive-site.xml.So HiveSerDe should fall back to hadoopconf if  hive.default.fileformat is not found in SQLConf

## How was this patch tested?

Tested manually.
Added UT

Closes #24489 from sandeep-katta/spark-27555.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 09:02:12 +09:00
gaoweikang 3859ca37d9 [SPARK-27586][SQL] Improve binary comparison: replace Scala's for-comprehension if statements with while loop
## What changes were proposed in this pull request?

This PR replaces for-comprehension if statement with while loop to gain better performance in `TypeUtils.compareBinary`.

## How was this patch tested?

Add UT to test old version and new version comparison result

Closes #24494 from woudygao/opt_binary_compare.

Authored-by: gaoweikang <gaoweikang@bytedance.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 20:33:27 -07:00
Marco Gaido 7a8cc8e071 [SPARK-27607][SQL] Improve Row.toString performance
## What changes were proposed in this pull request?

`Row.toString` is currently causing the useless creation of an `Array` containing all the values in the row before generating the string containing it. This operation adds a considerable overhead.

The PR proposes to avoid this operation in order to get a faster implementation.

## How was this patch tested?

Run

```scala
test("Row toString perf test") {
    val n = 100000
    val rows = (1 to n).map { i =>
      Row(i, i.toDouble, i.toString, i.toShort, true, null)
    }
    // warmup
    (1 to 10).foreach { _ => rows.foreach(_.toString) }

    val times = (1 to 100).map { _ =>
      val t0 = System.nanoTime()
      rows.foreach(_.toString)
      val t1 = System.nanoTime()
      t1 - t0
    }
    // scalastyle:off println
    println(s"Avg time on ${times.length} iterations for $n toString:" +
      s" ${times.sum.toDouble / times.length / 1e6} ms")
    // scalastyle:on println
  }
```
Before the PR:
```
Avg time on 100 iterations for 100000 toString: 61.08408419 ms
```
After the PR:
```
Avg time on 100 iterations for 100000 toString: 38.16539432 ms
```
This means the new implementation is about 1.60X faster than the original one.

Closes #24505 from mgaido91/SPARK-27607.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 07:20:33 -07:00
HyukjinKwon df8aa7ba8a [SPARK-27606][SQL] Deprecate 'extended' field in ExpressionDescription/ExpressionInfo
## What changes were proposed in this pull request?

After we added other fields, `arguments`, `examples`, `note` and `since` at SPARK-21485 and `deprecated` at SPARK-27328, we have nicer way to separately describe extended usages.

`extended` field and method at `ExpressionDescription`/`ExpressionInfo` is now pretty useless - it's not used in Spark side and only exists to keep backward compatibility.

This PR proposes to deprecate it.

## How was this patch tested?

Manually checked the deprecation waring is properly shown.

Closes #24500 from HyukjinKwon/SPARK-27606.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 21:10:00 +09:00
gatorsmile 2da406cae5 [SPARK-27618][SQL][FOLLOW-UP] Unnecessary access to externalCatalog
## What changes were proposed in this pull request?
This PR is to add test cases for ensuring that we do not have unnecessary access to externalCatalog.

In the future, we can follow these examples to improve our test coverage in this area.

## How was this patch tested?
N/A

Closes #24511 from gatorsmile/addTestcaseSpark-27618.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-01 20:09:46 -07:00
HyukjinKwon 3670826af6 [SPARK-26921][R][DOCS] Document Arrow optimization and vectorized R APIs
## What changes were proposed in this pull request?

This PR adds SparkR with Arrow optimization documentation.

Note that looks CRAN issue in Arrow side won't look likely fixed soon, IMHO, even after Spark 3.0.
If it happen to be fixed, I will fix this doc too later.

Another note is that Arrow R package itself requires R 3.5+. So, I intentionally didn't note this.

## How was this patch tested?

Manually built and checked.

Closes #24506 from HyukjinKwon/SPARK-26924.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 10:02:14 +09:00
Artem Kalchenko a35043c9e2 [SPARK-27591][SQL] Fix UnivocityParser for UserDefinedType
## What changes were proposed in this pull request?

Fix bug in UnivocityParser. makeConverter method didn't work correctly for UsedDefinedType

## How was this patch tested?

A test suite for UnivocityParser has been extended.

Closes #24496 from kalkolab/spark-27591.

Authored-by: Artem Kalchenko <artem.kalchenko@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-01 08:27:51 +09:00
Sean Owen 25ee0474f4 [SPARK-26936][MINOR][FOLLOWUP] Don't need the JobConf anymore, it seems
## What changes were proposed in this pull request?

On a second look in comments, seems like the JobConf isn't needed anymore here. It was used inconsistently before, it seems, and I don't see any reason a Hadoop Job config is required here anyway.

## How was this patch tested?

Existing tests.

Closes #24491 from srowen/SPARK-26936.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-29 19:47:20 -07:00
Wenchen Fan 7432e7ded4 [SPARK-24935][SQL][FOLLOWUP] support INIT -> UPDATE -> MERGE -> FINISH in Hive UDAF adapter
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24144 . #24144 missed one case: when hash aggregate fallback to sort aggregate, the life cycle of UDAF is: INIT -> UPDATE -> MERGE -> FINISH.

However, not all Hive UDAF can support it. Hive UDAF knows the aggregation mode when creating the aggregation buffer, so that it can create different buffers for different inputs: the original data or the aggregation buffer. Please see an example in the [sketches library](7f9e76e9e0/src/main/java/com/yahoo/sketches/hive/cpc/DataToSketchUDAF.java (L107)). The buffer for UPDATE may not support MERGE.

This PR updates the Hive UDAF adapter in Spark to support INIT -> UPDATE -> MERGE -> FINISH, by turning it to  INIT -> UPDATE -> FINISH + IINIT -> MERGE -> FINISH.

## How was this patch tested?

a new test case

Closes #24459 from cloud-fan/hive-udaf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-30 10:35:23 +08:00
Xiangrui Meng 618d6bff71 [SPARK-27588] Binary file data source fails fast and doesn't attempt to read very large files
## What changes were proposed in this pull request?

If a file is too big (>2GB), we should fail fast and do not try to read the file.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24483 from mengxr/SPARK-27588.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-29 16:24:49 -07:00
Gabor Somogyi fb6b19ab7c [SPARK-23014][SS] Fully remove V1 memory sink.
## What changes were proposed in this pull request?

There is a MemorySink v2 already so v1 can be removed. In this PR I've removed it completely.
What this PR contains:
* V1 memory sink removal
* V2 memory sink renamed to become the only implementation
* Since DSv2 sends exceptions in a chained format (linking them with cause field) I've made python side compliant
* Adapted all the tests

## How was this patch tested?

Existing unit tests.

Closes #24403 from gaborgsomogyi/SPARK-23014.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 09:44:23 -07:00
Sean Owen a6716d3f03 [SPARK-27571][CORE][YARN][EXAMPLES] Avoid scala.language.reflectiveCalls
## What changes were proposed in this pull request?

This PR avoids usage of reflective calls in Scala. It removes the import that suppresses the warnings and rewrites code in small ways to avoid accessing methods that aren't technically accessible.

## How was this patch tested?

Existing tests.

Closes #24463 from srowen/SPARK-27571.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:16:45 -05:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Liang-Chi Hsieh 76785cd6f0 [SPARK-27581][SQL] DataFrame countDistinct("*") shouldn't fail with AnalysisException
## What changes were proposed in this pull request?

Currently `countDistinct("*")` doesn't work. An analysis exception is thrown:

```scala
val df = sql("select id % 100 from range(100000)")
df.select(countDistinct("*")).first()

org.apache.spark.sql.AnalysisException: Invalid usage of '*' in expression 'count';
```

Users need to use `expr`.

```scala
df.select(expr("count(distinct(*))")).first()
```

This limits some API usage like `df.select(count("*"), countDistinct("*))`.

The PR takes the simplest fix that lets analyzer expand star and resolve `count` function.

## How was this patch tested?

Added unit test.

Closes #24482 from viirya/SPARK-27581.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 21:17:32 +08:00
Yuming Wang 5a62295219 [SPARK-27580][HOT-FIX] Fix wrong import order in FileScan.scala
## What changes were proposed in this pull request?
```
========================================================================
Running Scala style checks
========================================================================
[info] Checking Scala style using SBT with these profiles:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos
Scalastyle checks failed at following occurrences:
[error] /home/jenkins/workspace/SparkPullRequestBuilder/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala:29:0: org.apache.spark.sql.sources.Filter is in wrong order relative to org.apache.spark.sql.sources.v2.reader..
[error] Total time: 17 s, completed Apr 29, 2019 3:09:43 AM
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/104987/console

## How was this patch tested?
manual tests:
```
dev/scalastyle
```

Closes #24487 from wangyum/SPARK-27580.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 20:48:12 +08:00
Gengliang Wang 07d07fec03 [SPARK-27580][SQL] Implement doCanonicalize in BatchScanExec for comparing query plan results
## What changes were proposed in this pull request?

The method `QueryPlan.sameResult` is used for comparing logical plans in order to:
1. cache data in CacheManager
2. uncache data in CacheManager
3. Reuse subqueries
4. etc...

Currently the method `sameReuslt` always return false for `BatchScanExec`. We should fix it by implementing `doCanonicalize` for the node.

## How was this patch tested?

Unit test

Closes #24475 from gengliangwang/sameResultForV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 17:54:12 +08:00
Xiangrui Meng 20a3ef7259 [SPARK-27534][SQL] Do not load content column in binary data source if it is not selected
## What changes were proposed in this pull request?

A follow-up task from SPARK-25348. To save I/O cost, Spark shouldn't attempt to read the file if users didn't request the `content` column. For example:
```
spark.read.format("binaryFile").load(path).filter($"length" < 1000000).count()
```

## How was this patch tested?

Unit test added.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24473 from WeichenXu123/SPARK-27534.

Lead-authored-by: Xiangrui Meng <meng@databricks.com>
Co-authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-28 07:57:03 -07:00
Jash Gala 90085a1847 [SPARK-23619][DOCS] Add output description for some generator expressions / functions
## What changes were proposed in this pull request?

This PR addresses SPARK-23619: https://issues.apache.org/jira/browse/SPARK-23619

It adds additional comments indicating the default column names for the `explode` and `posexplode`
functions in Spark-SQL.

Functions for which comments have been updated so far:
* stack
* inline
* explode
* posexplode
* explode_outer
* posexplode_outer

## How was this patch tested?

This is just a change in the comments. The package builds and tests successfullly after the change.

Closes #23748 from jashgala/SPARK-23619.

Authored-by: Jash Gala <jashgala@amazon.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-27 10:30:12 +09:00
uncleGen 6328be78f9 [MINOR][TEST][DOC] Execute action miss name message
## What changes were proposed in this pull request?

some minor updates:
- `Execute` action miss `name` message
-  typo in SS document
-  typo in SQLConf

## How was this patch tested?

N/A

Closes #24466 from uncleGen/minor-fix.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-27 09:28:31 +08:00
Wenchen Fan 85fd552ed6 [SPARK-27190][SQL] add table capability for streaming
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24012 , to add the corresponding capabilities for streaming.

## How was this patch tested?

existing tests

Closes #24129 from cloud-fan/capability.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-26 15:44:23 +08:00
Wenchen Fan 2234667b15 [SPARK-27563][SQL][TEST] automatically get the latest Spark versions in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

We can get the latest downloadable Spark versions from https://dist.apache.org/repos/dist/release/spark/

## How was this patch tested?

manually.

Closes #24454 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-26 16:37:43 +09:00
Dongjoon Hyun d5dbf053d3 Revert "[SPARK-27439][SQL] Use analyzed plan when explaining Dataset"
This reverts commit ad60c6d9be.
2019-04-25 18:38:52 -07:00
Liang-Chi Hsieh 8b86326521 [SPARK-27551][SQL] Improve error message of mismatched types for CASE WHEN
## What changes were proposed in this pull request?

When there are mismatched types among cases or else values in case when expression, current error message is hard to read to figure out what and where the mismatch is.

This patch simply improves the error message for mismatched types for case when.

Before:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type;;
```

After:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN array<struct<x:bigint>> ELSE arr
ay<struct<y:bigint>> END;;
```

## How was this patch tested?

Added unit test.

Closes #24453 from viirya/SPARK-27551.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-25 08:47:19 -07:00
Wenchen Fan b7f9830670 [MINOR][TEST] switch from 2.4.1 to 2.4.2 in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

update `HiveExternalCatalogVersionsSuite` to test 2.4.2, as 2.4.1 will be removed from Mirror Network soon.

## How was this patch tested?

N/A

Closes #24452 from cloud-fan/release.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-25 10:26:40 +08:00
gatorsmile cd4a284030 [SPARK-27460][FOLLOW-UP][TESTS] Fix flaky tests
## What changes were proposed in this pull request?

This patch makes several test flakiness fixes.

## How was this patch tested?
N/A

Closes #24434 from gatorsmile/fixFlakyTest.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-24 17:36:29 +08:00
HyukjinKwon a30983db57 [SPARK-27512][SQL] Avoid to replace ',' in CSV's decimal type inference for backward compatibility
## What changes were proposed in this pull request?

The code below currently infers as decimal but previously it was inferred as string.

**In branch-2.4**, type inference path for decimal and parsing data are different.

2a8343121e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala (L153)

c284c4e1f6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala (L125)

So the code below:

```scala
scala> spark.read.option("delimiter", "|").option("inferSchema", "true").csv(Seq("1,2").toDS).printSchema()
```

produced string as its type.

```
root
 |-- _c0: string (nullable = true)
```

**In the current master**, it now infers decimal as below:

```
root
 |-- _c0: decimal(2,0) (nullable = true)
```

It happened after https://github.com/apache/spark/pull/22979 because, now after this PR, we only have one way to parse decimal:

7a83d71403/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala (L92)

**After the fix:**

```
root
 |-- _c0: string (nullable = true)
```

This PR proposes to restore the previous behaviour back in `CSVInferSchema`.

## How was this patch tested?

Manually tested and unit tests were added.

Closes #24437 from HyukjinKwon/SPARK-27512.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 16:22:07 +09:00
Sean Owen 596a5ff273 [MINOR][BUILD] Update genjavadoc to 0.13
## What changes were proposed in this pull request?

Kind of related to https://github.com/gatorsmile/spark/pull/5 - let's update genjavadoc to see if it generates fewer spurious javadoc errors to begin with.

## How was this patch tested?

Existing docs build

Closes #24443 from srowen/genjavadoc013.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 13:44:48 +09:00
Gengliang Wang 00f2f311f7 [SPARK-27128][SQL] Migrate JSON to File Data Source V2
## What changes were proposed in this pull request?
Migrate JSON to File Data Source V2

## How was this patch tested?

Unit test

Closes #24058 from gengliangwang/jsonV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-23 22:39:59 +08:00
pengbo d9b2ce0f0f [SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values
## What changes were proposed in this pull request?
This PR is follow up of https://github.com/apache/spark/pull/24286. As gatorsmile pointed out that column with null value is inaccurate as well.

```
> select key from test;
2
NULL
1
spark-sql> desc extended test key;
col_name key
data_type int
comment NULL
min 1
max 2
num_nulls 1
distinct_count 2
```

The distinct count should be distinct_count + 1 when column contains null value.
## How was this patch tested?

Existing tests & new UT added.

Closes #24436 from pengbo/aggregation_estimation.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-22 20:30:08 -07:00
Maxim Gekk 93a264d05a [SPARK-27535][SQL][TEST] Date and timestamp JSON benchmarks
## What changes were proposed in this pull request?

Added new JSON benchmarks related to date and timestamps operations:
- Write date/timestamp to JSON files
- `to_json()` and `from_json()` for dates and timestamps
- Read date/timestamps from JSON files, and infer schemas
- Parse and infer schemas from `Dataset[String]`

Also existing JSON benchmarks are ported on `NoOp` datasource.

Closes #24430 from MaxGekk/json-datetime-benchmark.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:09:14 +09:00
Maxim Gekk 55f26d8090 [SPARK-27533][SQL][TEST] Date and timestamp CSV benchmarks
## What changes were proposed in this pull request?

Added new CSV benchmarks related to date and timestamps operations:
- Write date/timestamp to CSV files
- `to_csv()` and `from_csv()` for dates and timestamps
- Read date/timestamps from CSV files, and infer schemas
- Parse and infer schemas from `Dataset[String]`

Also existing CSV benchmarks are ported on `NoOp` datasource.

Closes #24429 from MaxGekk/csv-timestamp-benchmark.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:08:02 +09:00
Maxim Gekk 43a73e387c [SPARK-27528][SQL] Use Parquet logical type TIMESTAMP_MICROS by default
## What changes were proposed in this pull request?

In the PR, I propose to use the `TIMESTAMP_MICROS` logical type for timestamps written to parquet files. The type matches semantically to Catalyst's `TimestampType`, and stores microseconds since epoch in UTC time zone. This will allow to avoid conversions of microseconds to nanoseconds and to Julian calendar. Also this will reduce sizes of written parquet files.

## How was this patch tested?

By existing test suites.

Closes #24425 from MaxGekk/parquet-timestamp_micros.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:06:39 +09:00
Dilip Biswal 3240e52dc7 [SPARK-27531][SQL] Improve EXPLAIN DESC TABLE to show the input parameters of the command.
## What changes were proposed in this pull request?
Currently "EXPLAIN DESC TABLE" is special cased and outputs a single row relation as following.
Current output:
```sql
spark-sql> EXPLAIN DESCRIBE TABLE t;
== Physical Plan ==
*(1) Scan OneRowRelation[]
```
This is not consistent with how we handle explain processing for other commands. In this PR, the inconsistency is handled by removing the special handling for "describe table".

After change:
```sql
spark-sql> EXPLAIN DESC EXTENDED t
== Physical Plan ==
Execute DescribeTableCommand
   +- DescribeTableCommand `t`, true
```
## How was this patch tested?
Added new tests in SQLQueryTestSuite.

Closes #24427 from dilipbiswal/describe_table_explain2.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-22 13:02:10 -07:00
Eric Liang 5172190da1 [SPARK-27392][SQL] TestHive test tables should be placed in shared test state, not per session
## What changes were proposed in this pull request?

Otherwise, tests that use tables from multiple sessions will run into issues if they access the same table. The correct location is in shared state.

A couple other minor test improvements.

cc gatorsmile srinathshankar

## How was this patch tested?

Existing unit tests.

Closes #24302 from ericl/test-conflicts.

Lead-authored-by: Eric Liang <ekl@databricks.com>
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-22 11:05:31 -07:00
Maxim Gekk 79d3bc0409 [SPARK-27438][SQL] Parse strings with timestamps by to_timestamp() in microsecond precision
## What changes were proposed in this pull request?

In the PR, I propose to parse strings to timestamps in microsecond precision by the ` to_timestamp()` function if the specified pattern contains a sub-pattern for seconds fractions.

Closes #24342

## How was this patch tested?

By `DateFunctionsSuite` and `DateExpressionsSuite`

Closes #24420 from MaxGekk/to_timestamp-microseconds3.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-22 19:41:32 +08:00
Maxim Gekk 777b797867 [SPARK-27522][SQL][TEST] Test migration from INT96 to TIMESTAMP_MICROS for timestamps in parquet
## What changes were proposed in this pull request?

Added tests to check migration from `INT96` to `TIMESTAMP_MICROS` (`INT64`) for timestamps in parquet files. In particular:
- Append `TIMESTAMP_MICROS` timestamps to **existing parquet** files with `INT96` timestamps
- Append `TIMESTAMP_MICROS` timestamps to a table with `INT96` timestamps
- Append `INT96` to `TIMESTAMP_MICROS` timestamps in **parquet files**
- Append `INT96` to `TIMESTAMP_MICROS` timestamps in a **table**

Closes #24417 from MaxGekk/parquet-timestamp-int64-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-22 16:34:13 +09:00
Dilip Biswal 8a8643c28d [SPARK-27480][SQL] Improve EXPLAIN DESC QUERY to show the input SQL statement
Currently running explain on describe query gives a little confusing output. This is a minor pr that improves the output of explain.

Before
```
1.EXPLAIN DESCRIBE WITH s AS (SELECT 'hello' as col1) SELECT * FROM s;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand CTE [s]
2.EXPLAIN EXTENDED DESCRIBE SELECT * from s1 where c1 > 0;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand 'Project [*]
```
After
```
1. EXPLAIN DESCRIBE WITH s AS (SELECT 'hello' as col1) SELECT * FROM s;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s
2. EXPLAIN DESCRIBE SELECT * from s1 where c1 > 0;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand SELECT * from s1 where c1 > 0
```
Added a couple of tests in describe-query.sql under SQLQueryTestSuite.

Closes #24385 from dilipbiswal/describe_query_explain.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 15:35:05 -07:00
WeichenXu 9793d9ec22 [SPARK-27473][SQL] Support filter push down for status fields in binary file data source
## What changes were proposed in this pull request?

Support 4 kinds of filters:
- LessThan
- LessThanOrEqual
- GreatThan
- GreatThanOrEqual

Support filters applied on 2 columns:
- modificationTime
- length

Note:
In order to support datasource filter push-down, I flatten schema to be:
```
val schema = StructType(
    StructField("path", StringType, false) ::
    StructField("modificationTime", TimestampType, false) ::
    StructField("length", LongType, false) ::
    StructField("content", BinaryType, true) :: Nil)
```

## How was this patch tested?

To be added.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24387 from WeichenXu123/binary_ds_filter.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-21 12:45:59 -07:00
Liang-Chi Hsieh ad60c6d9be [SPARK-27439][SQL] Use analyzed plan when explaining Dataset
## What changes were proposed in this pull request?

Because a review is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain
```

Before:
```scala
== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)

```

## How was this patch tested?

Manually test and unit test.

Closes #24415 from viirya/SPARK-27439.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 10:25:56 -07:00
Maxim Gekk d61b3bc875 [SPARK-27527][SQL][DOCS] Improve descriptions of Timestamp and Date types
## What changes were proposed in this pull request?

In the PR, I propose more precise description of `TimestampType` and `DateType`, how they store timestamps and dates internally.

Closes #24424 from MaxGekk/timestamp-date-type-doc.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-21 16:53:11 +09:00
Gengliang Wang 31488e1ca5 [SPARK-27504][SQL] File source V2: support refreshing metadata cache
## What changes were proposed in this pull request?

In file source V1, if some file is deleted manually, reading the DataFrame/Table will throws an exception with suggestion message
```
It is possible the underlying files have been updated. You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved.
```
After refreshing the table/DataFrame, the reads should return correct results.

We should follow it in file source V2 as well.
## How was this patch tested?
Unit test

Closes #24401 from gengliangwang/refreshFileTable.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-19 18:26:03 +08:00
Yifei Huang 163a6e2982 [SPARK-27514] Skip collapsing windows with empty window expressions
## What changes were proposed in this pull request?

A previous change moved the removal of empty window expressions to the RemoveNoopOperations rule, which comes after the CollapseWindow rule. Therefore, by the time we get to CollapseWindow, we aren't guaranteed that empty windows have been removed. This change checks that the window expressions are not empty, and only collapses the windows if both windows are non-empty.

A lengthier description and repro steps here: https://issues.apache.org/jira/browse/SPARK-27514

## How was this patch tested?

A unit test, plus I reran the breaking case mentioned in the Jira ticket.

Closes #24411 from yifeih/yh/spark-27514.

Authored-by: Yifei Huang <yifeih@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-19 14:04:44 +08:00
Yuming Wang 8f82237a5b [SPARK-27501][SQL][TEST] Add test for HIVE-13083: Writing HiveDecimal to ORC can wrongly suppress present stream
## What changes were proposed in this pull request?

This PR add test for [HIVE-13083](https://issues.apache.org/jira/browse/HIVE-13083): Writing HiveDecimal to ORC can wrongly suppress present stream.

## How was this patch tested?
manual tests:
```
build/sbt  "hive/testOnly *HiveOrcQuerySuite" -Phive -Phadoop-3.2
```

Closes #24397 from wangyum/SPARK-26437.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-19 10:12:21 +09:00
Gengliang Wang 3748b381df [SPARK-27460][TESTS][FOLLOWUP] Add HiveClientVersions to parallel test suite list
## What changes were proposed in this pull request?

The test time of `HiveClientVersions` is around 3.5 minutes.
This PR is to add it into the parallel test suite list. To make sure there is no colliding warehouse location,  we can change the warehouse path to a temporary directory.

## How was this patch tested?

Unit test

Closes #24404 from gengliangwang/parallelTestFollowUp.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-18 15:37:55 -07:00
Liang-Chi Hsieh 9c41bfd83c [SPARK-27502][SQL][TEST] Update nested schema benchmark result for Orc V2
## What changes were proposed in this pull request?

We added nested schema pruning support to Orc V2 recently. The benchmark result should be updated. The benchmark numbers are obtained by running benchmark on r3.xlarge machine.

## How was this patch tested?

Test only change.

Closes #24399 from viirya/update-orcv2-benchmark.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-18 08:08:22 -07:00
Gengliang Wang 9c238b8a46 [SPARK-27460][TESTS] Running slowest test suites in their own forked JVMs for higher parallelism
## What changes were proposed in this pull request?

This patch modifies SparkBuild so that the largest / slowest test suites (or collections of suites) can run in their own forked JVMs, allowing them to be run in parallel with each other. This opt-in / whitelisting approach allows us to increase parallelism without having to fix a long-tail of flakiness / brittleness issues in tests which aren't performance bottlenecks.

See comments in SparkBuild.scala for information on the details, including a summary of why we sometimes opt to run entire groups of tests in a single forked JVM .

The time of full new pull request test in Jenkins is reduced by around 53%:
before changes: 4hr 40min
after changes: 2hr 13min

## How was this patch tested?

Unit test

Closes #24373 from gengliangwang/parallelTest.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 20:49:36 +08:00
Gengliang Wang 7d44ba05d1 [SPARK-27490][SQL] File source V2: return correct result for Dataset.inputFiles()
## What changes were proposed in this pull request?

Currently, a `Dateset` with file source V2 always return empty results for method `Dataset.inputFiles()`.

We should fix it.

## How was this patch tested?

Unit test

Closes #24393 from gengliangwang/inputFiles.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 14:39:30 +08:00
Kris Mok 50bdc9befa [SPARK-27423][SQL][FOLLOWUP] Minor polishes to Cast codegen templates for Date <-> Timestamp
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/24332 introduced an unnecessary `import` statement and two slight issues in the codegen templates in `Cast` for `Date` <-> `Timestamp`.
This PR removes the unused import statement and fixes the slight codegen issue.

The issue in those two codegen templates is this pattern:
```scala
val zid = JavaCode.global(
  ctx.addReferenceObj("zoneId", zoneId, "java.time.ZoneId"),
  zoneId.getClass)
```
`zoneId` can refer to an instance of a non-public class, e.g. `java.time.ZoneRegion`, and while this code correctly puts in the 3rd argument to `ctx.addReferenceObj()`, it's still passing `zoneId.getClass` to `JavaCode.global()` which is not desirable, but doesn't cause any immediate bugs in this particular case, because `zid` is used in an expression immediately afterwards.
If this `zid` ever needs to spill to any explicitly typed variables, e.g. a local variable, and if the spill handling uses the `javaType` on this `GlobalVariable`, it'd generate code that looks like:
```java
java.time.ZoneRegion value1 = ((java.time.ZoneId) references[2] /* literal */);
```
which would then be a real bug:
- a non-accessible type `java.time.ZoneRegion` is referenced in the generated code, and
- `ZoneId` -> `ZoneRegion` requires an explicit downcast.

## How was this patch tested?

Existing tests. This PR does not change behavior, and the original PR won't cause any real behavior bug to begin with.

Closes #24392 from rednaxelafx/spark-27423-followup.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 14:27:33 +08:00
Dilip Biswal e1c90d66bb [SPARK-19712][SQL] Pushdown LeftSemi/LeftAnti below join
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below the Join operator.
This is a prerequisite work thats needed for the subsequent task of moving the subquery rewrites to the beginning of optimization phase.

The larger  PR is [here](https://github.com/apache/spark/pull/23211) . This PR addresses the comment at [link](https://github.com/apache/spark/pull/23211#issuecomment-445705922).
## How was this patch tested?
Added tests under LeftSemiAntiJoinPushDownSuite.

Closes #24331 from dilipbiswal/SPARK-19712-pushleftsemi-belowjoin.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 20:30:20 +08:00
Wenchen Fan e6618de809 [SPARK-27430][SQL] broadcast hint should be respected for broadcast nested loop join
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/24164

broadcast hint should be respected for broadcast nested loop join. This PR also refactors the related code a little bit, to save duplicated code.

## How was this patch tested?

new tests

Closes #24376 from cloud-fan/join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 19:29:28 +08:00
pengbo 54b0d1e0ef [SPARK-27416][SQL] UnsafeMapData & UnsafeArrayData Kryo serialization …
## What changes were proposed in this pull request?
Finish the rest work of https://github.com/apache/spark/pull/24317, https://github.com/apache/spark/pull/9030
a. Implement Kryo serialization for UnsafeArrayData
b. fix UnsafeMapData Java/Kryo Serialization issue when two machines have different Oops size
c. Move the duplicate code "getBytes()" to Utils.

## How was this patch tested?
According Units has been added & tested

Closes #24357 from pengbo/SPARK-27416_new.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 13:03:00 +08:00
WeichenXu 1bb0c8e407 [SPARK-25348][SQL] Data source for binary files
## What changes were proposed in this pull request?

Implement binary file data source in Spark.

Format name: "binaryFile" (case-insensitive)

Schema:
- content: BinaryType
- status: StructType
  - path: StringType
  - modificationTime: TimestampType
  - length: LongType

Options:
* pathGlobFilter (instead of pathFilterRegex) to reply on GlobFilter behavior
* maxBytesPerPartition is not implemented since it is controlled by two SQL confs: maxPartitionBytes and openCostInBytes.

## How was this patch tested?

Unit test added.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24354 from WeichenXu123/binary_file_datasource.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-16 15:41:32 -07:00
liwensun 26ed65f415 [SPARK-27453] Pass partitionBy as options in DataFrameWriter
## What changes were proposed in this pull request?

Pass partitionBy columns as options and feature-flag this behavior.

## How was this patch tested?

A new unit test.

Closes #24365 from liwensun/partitionby.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2019-04-16 15:03:16 -07:00
Liang-Chi Hsieh b404e02574 [SPARK-27476][SQL] Refactoring SchemaPruning rule to remove duplicate code
## What changes were proposed in this pull request?

In SchemaPruning rule, there is duplicate code for data source v1 and v2. Their logic is the same and we can refactor the rule to remove duplicate code.

## How was this patch tested?

Existing tests.

Closes #24383 from viirya/SPARK-27476.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-16 14:50:37 -07:00
shivusondur 88d9de26dd [SPARK-27464][CORE] Added Constant instead of referring string literal used from many places
## What changes were proposed in this pull request?

Added Constant instead of referring the same String literal "spark.buffer.pageSize" from many places
## How was this patch tested?
Run the corresponding Unit Test Cases manually.

Closes #24368 from shivusondur/Constant.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:30:46 -05:00
Gengliang Wang f9837d3bf6 [SPARK-27448][SQL] File source V2 table provider should be compatible with V1 provider
## What changes were proposed in this pull request?

In the rule `PreprocessTableCreation`, if an existing table is appended with a different provider, the action will fail.
Currently, there are two implementations for file sources and creating a table with file source V2 will always fall back to V1 FileFormat. We should consider the following cases as valid:
1. Appending a table with file source V2 provider using the v1 file format
2. Appending a table with v1 file format provider using file source V2 format

## How was this patch tested?

Unit test

Closes #24356 from gengliangwang/fixTableProvider.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-16 14:26:38 +08:00
pengbo c58a4fed8d [SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…
## What changes were proposed in this pull request?
The upper bound of group-by columns row number is to multiply distinct counts of group-by columns. However, column with only null value will cause the output row number to be 0 which is incorrect.
Ex:
col1 (distinct: 2, rowCount 2)
col2 (distinct: 0, rowCount 2)
=> group by col1, col2
Actual: output rows: 0
Expected: output rows: 2

## How was this patch tested?
According unit test has been added, plus manual test has been done in our tpcds benchmark environement.

Closes #24286 from pengbo/master.

Lead-authored-by: pengbo <bo.peng1019@gmail.com>
Co-authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-15 15:37:07 -07:00
Dilip Biswal 3ab96d7acf [SPARK-27444][SQL][FOLLOWUP][MINOR][TEST] Add a test for describing multi select query.
## What changes were proposed in this pull request?
This is a minor pr to add a test to describe a multi select query.

## How was this patch tested?
Added a test in describe-query.sql

Closes #24370 from dilipbiswal/describe-query-multiselect-test.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-15 21:26:45 +08:00
Gengliang Wang 27d625d785 [SPARK-27459][SQL] Revise the exception message of schema inference failure in file source V2
## What changes were proposed in this pull request?

Since https://github.com/apache/spark/pull/23383/files#diff-db4a140579c1ac4b1dbec7fe5057eecaR36, the exception message of schema inference failure in file source V2 is `tableName`, which is equivalent to `shortName + path`.

While in file source V1, the message is `Unable to infer schema from ORC/CSV/JSON...`.
We should make the message in V2 consistent with V1, so that in the future migration the related test cases don't need to be modified. https://github.com/apache/spark/pull/24058#pullrequestreview-226364350

## How was this patch tested?

Revert the modified unit test cases in https://github.com/apache/spark/pull/24005/files#diff-b9ddfbc9be8d83ecf100b3b8ff9610b9R431 and https://github.com/apache/spark/pull/23383/files#diff-9ab56940ee5a53f2bb81e3c008653362R577, and test with them.

Closes #24369 from gengliangwang/reviseInferSchemaMessage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-15 21:06:03 +08:00
herman 4704af4c26 [SPARK-27449] Move WholeStageCodegen.limitNotReachedCond class checks into separate methods.
## What changes were proposed in this pull request?
This PR moves the checks done in `WholeStageCodegen.limitNotReachedCond` into a separate protected method. This makes it easier to introduce new leaf or blocking nodes.

## How was this patch tested?
Existing tests.

Closes #24358 from hvanhovell/SPARK-27449.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-14 15:54:20 +08:00
Wenchen Fan 0407070945 [SPARK-27444][SQL] multi-select can be used in subquery
## What changes were proposed in this pull request?

This is a regression caused by https://github.com/apache/spark/pull/24150

`select * from (from a select * select *)` is supported in 2.4, and we should keep supporting it.

This PR merges the parser rule for single and multi select statements, as they are very similar.

## How was this patch tested?

a new test case

Closes #24348 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 20:57:34 +08:00
Gengliang Wang 4eb694c58f [SPARK-27443][SQL] Support UDF input_file_name in file source V2
## What changes were proposed in this pull request?

Currently, if we select the UDF `input_file_name` as a column in file source V2, the results are empty.
We should support it in file source V2.

## How was this patch tested?

Unit test

Closes #24347 from gengliangwang/input_file_name.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 20:30:42 +08:00
Kris Mok bbbe54aa79 [SPARK-27199][SQL][FOLLOWUP] Fix bug in codegen templates in UnixTime and FromUnixTime
## What changes were proposed in this pull request?

SPARK-27199 introduced the use of `ZoneId` instead of `TimeZone` in a few date/time expressions.
There were 3 occurrences of `ctx.addReferenceObj("zoneId", zoneId)` in that PR, which had a bug because while the `java.time.ZoneId` base type is public, the actual concrete implementation classes are not public, so using the 2-arg version of `CodegenContext.addReferenceObj` would incorrectly generate code that reference non-public types (`java.time.ZoneRegion`, to be specific). The 3-arg version should be used, with the class name of the referenced object explicitly specified to the public base type.

One of such occurrences was caught in testing in the main PR of SPARK-27199 (https://github.com/apache/spark/pull/24141), for `DateFormatClass`. But the other 2 occurrences slipped through because there were no test cases that covered them.

Example of this bug in the current Apache Spark master, in a Spark Shell:
```
scala> Seq(("2016-04-08", "yyyy-MM-dd")).toDF("s", "f").repartition(1).selectExpr("to_unix_timestamp(s, f)").show
...
java.lang.IllegalAccessError: tried to access class java.time.ZoneRegion from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1
```

This PR fixes the codegen issues and adds the corresponding unit tests.

## How was this patch tested?

Enhanced tests in `DateExpressionsSuite` for `to_unix_timestamp` and `from_unixtime`.

Closes #24352 from rednaxelafx/fix-spark-27199.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 13:31:18 +08:00
Dilip Biswal 5d8aee5886 [SPARK-27445][SQL][TEST] Update SQLQueryTestSuite to process files ending with .sql
## What changes were proposed in this pull request?
While using vi or vim to edit the test files the .swp or .swo files are created and attempt to run the test suite in the presence of these files causes errors like below :
```
nfo] - subquery/exists-subquery/.exists-basic.sql.swp *** FAILED *** (117 milliseconds)
[info]   java.io.FileNotFoundException: /Users/dbiswal/mygit/apache/spark/sql/core/target/scala-2.12/test-classes/sql-tests/results/subquery/exists-subquery/.exists-basic.sql.swp.out (No such file or directory)
[info]   at java.io.FileInputStream.open0(Native Method)
[info]   at java.io.FileInputStream.open(FileInputStream.java:195)
[info]   at java.io.FileInputStream.<init>(FileInputStream.java:138)
[info]   at org.apache.spark.sql.catalyst.util.package$.fileToString(package.scala:49)
[info]   at org.apache.spark.sql.SQLQueryTestSuite.runQueries(SQLQueryTestSuite.scala:247)
[info]   at org.apache.spark.sql.SQLQueryTestSuite.$anonfun$runTest$11(SQLQueryTestSuite.scala:192)
```
~~This minor pr adds these temp files in the ignore list.~~
While computing the list of test files to process, only consider files with `.sql` extension. This makes sure the unwanted temp files created from various editors are ignored from processing.
## How was this patch tested?
Verified manually.

Closes #24333 from dilipbiswal/dkb_sqlquerytest.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 14:50:46 -07:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
maryannxue 43da473c1c [SPARK-27225][SQL] Implement join strategy hints
## What changes were proposed in this pull request?

This PR extends the existing BROADCAST join hint (for both broadcast-hash join and broadcast-nested-loop join) by implementing other join strategy hints corresponding to the rest of Spark's existing join strategies: shuffle-hash, sort-merge, cartesian-product. The hint names: SHUFFLE_MERGE, SHUFFLE_HASH, SHUFFLE_REPLICATE_NL are partly different from the code names in order to make them clearer to users and reflect the actual algorithms better.

The hinted strategy will be used for the join with which it is associated if it is applicable/doable.

Conflict resolving rules in case of multiple hints:
1. Conflicts within either side of the join: take the first strategy hint specified in the query, or the top hint node in Dataset. For example, in "select /*+ merge(t1) */ /*+ broadcast(t1) */ k1, v2 from t1 join t2 on t1.k1 = t2.k2", take "merge(t1)"; in ```df1.hint("merge").hint("shuffle_hash").join(df2)```, take "shuffle_hash". This is a general hint conflict resolving strategy, not specific to join strategy hint.
2. Conflicts between two sides of the join:
  a) In case of different strategy hints, hints are prioritized as ```BROADCAST``` over ```SHUFFLE_MERGE``` over ```SHUFFLE_HASH``` over ```SHUFFLE_REPLICATE_NL```.
  b) In case of same strategy hints but conflicts in build side, choose the build side based on join type and size.

## How was this patch tested?

Added new UTs.

Closes #24164 from maryannxue/join-hints.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 00:14:37 +08:00
s71955 239082d966 [SPARK-27403][SQL] Fix updateTableStats to update table stats always with new stats or None
## What changes were proposed in this pull request?

System shall update the table stats automatically if user set spark.sql.statistics.size.autoUpdate.enabled as true, currently this property is not having any significance even if it is enabled or disabled. This feature is similar to Hives auto-gather feature where statistics are automatically computed by default if this feature is enabled.
Reference:
https://cwiki.apache.org/confluence/display/Hive/StatsDev

As part of fix , autoSizeUpdateEnabled  validation is been done initially so that system will calculate the table size for the user automatically and record it in metastore as per user expectation.

## How was this patch tested?
UT is written and manually verified in cluster.
Tested with unit tests + some internal tests on real cluster.

Before fix:

![image](https://user-images.githubusercontent.com/12999161/55688682-cd8d4780-5998-11e9-85da-e1a4e34419f6.png)

After fix
![image](https://user-images.githubusercontent.com/12999161/55688654-7d15ea00-5998-11e9-973f-1f4cee27018f.png)

Closes #24315 from sujith71955/master_autoupdate.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 08:53:00 -07:00
Gengliang Wang 4177292dcd [SPARK-27435][SQL] Support schema pruning in ORC V2
## What changes were proposed in this pull request?

Currently, the optimization rule `SchemaPruning` only works for Parquet/Orc V1.
We should have the same optimization in ORC V2.

## How was this patch tested?

Unit test

Closes #24338 from gengliangwang/schemaPruningForV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-11 20:03:32 +08:00
chakravarthiT 074533334d [SPARK-27088][SQL] Add a configuration to set log level for each batch at RuleExecutor
## What changes were proposed in this pull request?

Similar to #22406 , which has made log level for plan changes by each rule configurable ,this PR is to make log level for plan changes by each batch configurable,and I have reused the same configuration: "spark.sql.optimizer.planChangeLog.level".

Config proposed in this PR ,
spark.sql.optimizer.planChangeLog.batches - enable plan change logging only for a set of specified batches, separated by commas.

## How was this patch tested?

Added UT , also tested manually and attached screenshots below.

1)Setting spark.sql.optimizer.planChangeLog.leve to warn.

![settingLogLevelToWarn](https://user-images.githubusercontent.com/45845595/54556730-8803dd00-49df-11e9-95ab-ebb0c8d735ef.png)

2)setting spark.sql.optimizer.planChangeLog.batches to Resolution and Subquery.
![settingBatchestoLog](https://user-images.githubusercontent.com/45845595/54556740-8cc89100-49df-11e9-80ab-fbbbe1ff2cdf.png)

3)  plan change logging enabled only for a set of specified batches(Resolution and Subquery)
![batchloggingOp](https://user-images.githubusercontent.com/45845595/54556788-ab2e8c80-49df-11e9-9ae0-57815f552896.png)

Closes #24136 from chakravarthiT/logBatches.

Lead-authored-by: chakravarthiT <45845595+chakravarthiT@users.noreply.github.com>
Co-authored-by: chakravarthiT <tcchakra@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:02:27 +09:00
ocaballero 181d190c60 [MINOR][SQL] Unnecessary access to externalCatalog
Necessarily access the external catalog without having to do it

## What changes were proposed in this pull request?

The existsFunction function has been changed because it unnecessarily accessed the externalCatalog to find if the database exists in cases where the function is in the functionRegistry

## How was this patch tested?

It has been tested through spark-shell and accessing the metastore logs of hive.

Inside spark-shell we use spark.table (% tableA%). SelectExpr ("trim (% columnA%)") in the current version and it appears every time:

org.apache.hadoop.hive.metastore.HiveMetaStore.audit: cmd = get_database: default

Once the change is made, no record appears

Closes #24312 from OCaballero/master.

Authored-by: ocaballero <oliver.caballero.alvarez@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:00:09 +09:00
Maxim Gekk ab8710b579 [SPARK-27423][SQL] Cast DATE <-> TIMESTAMP according to the SQL standard
## What changes were proposed in this pull request?

According to SQL standard, value of `DATE` type is union of year, month, dayInMonth, and it is independent from any time zones. To convert it to Catalyst's `TIMESTAMP`, `DATE` value should be "extended" by the time at midnight - `00:00:00`. The resulted local date+time should be considered as a timestamp in the session time zone, and casted to microseconds since epoch in `UTC` accordingly.

The reverse casting from `TIMESTAMP` to `DATE` should be performed in the similar way. `TIMESTAMP` values should be represented as a local date+time in the session time zone. And the time component should be just removed. For example, `TIMESTAMP 2019-04-10 00:10:12` -> `DATE 2019-04-10`. The resulted date is converted to days since epoch `1970-01-01`.

## How was this patch tested?

The changes were tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`.

Closes #24332 from MaxGekk/cast-timestamp-to-date2.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 22:41:19 +08:00
Maxim Gekk 1470f23ec9 [SPARK-27422][SQL] current_date() should return current date in the session time zone
## What changes were proposed in this pull request?

In the PR, I propose to revert 2 commits 06abd06112 and 61561c1c2d, and take current date via `LocalDate.now` in the session time zone. The result is stored as days since epoch `1970-01-01`.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite`, and `ComputeCurrentTimeSuite`.

Closes #24330 from MaxGekk/current-date2.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 21:54:50 +08:00
10129659 5ea4deec44 [SPARK-26012][SQL] Null and '' values should not cause dynamic partition failure of string types
Dynamic partition will fail when both '' and null values are taken as dynamic partition values simultaneously.
For example, the test bellow will fail before this PR:

test("Null and '' values should not cause dynamic partition failure of string types") {
withTable("t1", "t2") {
spark.range(3).write.saveAsTable("t1")
spark.sql("select id, cast(case when id = 1 then '' else null end as string) as p" +
" from t1").write.partitionBy("p").saveAsTable("t2")
checkAnswer(spark.table("t2").sort("id"), Seq(Row(0, null), Row(1, null), Row(2, null)))
}
}

The error is: 'org.apache.hadoop.fs.FileAlreadyExistsException: File already exists'.
This PR convert the empty strings to null for partition values.
This is another way for PR(https://github.com/apache/spark/pull/23010)

(Please fill in changes proposed in this fix)

How was this patch tested?
New added test.

Closes #24334 from eatoncys/FileFormatWriter.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 19:54:19 +08:00
韩田田00222924 85e5d4f141 [SPARK-24872] Replace taking the $symbol with $sqlOperator in BinaryOperator's toString method
## What changes were proposed in this pull request?

For BinaryOperator's toString method, it's better to use `$sqlOperator` instead of `$symbol`.

## How was this patch tested?

We can test this patch  with unit tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #21826 from httfighter/SPARK-24872.

Authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:58:01 +08:00
Wenchen Fan 2e90574dd0 [SPARK-27414][SQL] make it clear that date type is timezone independent
## What changes were proposed in this pull request?

In SQL standard, date type is a union of the `year`, `month` and `day` fields. It's timezone independent, which means it does not represent a specific point in the timeline.

Spark SQL follows the SQL standard, this PR is to make it clear that date type is timezone independent
1. improve the doc to highlight that date is timezone independent.
2. when converting string to date,  uses the java time API that can directly parse a `LocalDate` from a string, instead of converting `LocalDate` to a `Instant` at UTC first.
3. when converting date to string, uses the java time API that can directly format a `LocalDate` to a string, instead of converting `LocalDate` to a `Instant` at UTC first.

2 and 3 should not introduce any behavior changes.

## How was this patch tested?

existing tests

Closes #24325 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:39:28 +08:00
Ryan Blue 58674d54ba [SPARK-27181][SQL] Add public transform API
## What changes were proposed in this pull request?

This adds a public Expression API that can be used to pass partition transformations to data sources.

## How was this patch tested?

Existing tests to validate no regressions. Added transform cases to DDL suite and v1 conversions suite.

Closes #24117 from rdblue/add-public-transform-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 14:30:39 +08:00
Liang-Chi Hsieh 08858f6abc [SPARK-27253][SQL][FOLLOW-UP] Update doc about parent-session configuration priority
## What changes were proposed in this pull request?

The PR #24189 changes the behavior of merging SparkConf. The existing doc is not updated for it. This is a followup of it to update the doc.

## How was this patch tested?

Doc only change.

Closes #24326 from viirya/SPARK-27253-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-10 13:21:21 +09:00
Maxim Gekk 63e4bf42c2 [SPARK-27401][SQL] Refactoring conversion of Timestamp to/from java.sql.Timestamp
## What changes were proposed in this pull request?

In the PR, I propose simpler implementation of `toJavaTimestamp()`/`fromJavaTimestamp()` by reusing existing functions of `DateTimeUtils`. This will allow to:
- Simply implementation of `toJavaTimestamp()`, and handle properly negative inputs.
- Detect `Long` overflow in conversion of milliseconds (`java.sql.Timestamp`) to microseconds (Catalyst's Timestamp).

## How was this patch tested?

By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`. And by new benchmark for export/import timestamps added to `DateTimeBenchmark`:

Before:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             290            335          49         17.2          58.0       1.0X
Collect longs                                      1234           1681         487          4.1         246.8       0.2X
Collect timestamps                                 1718           1755          63          2.9         343.7       0.2X
```

After:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             283            301          19         17.7          56.6       1.0X
Collect longs                                      1048           1087          36          4.8         209.6       0.3X
Collect timestamps                                 1425           1479          56          3.5         285.1       0.2X
```

Closes #24311 from MaxGekk/conv-java-sql-date-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-09 15:42:27 -07:00
Gengliang Wang 3db117e43e [SPARK-27407][SQL] File source V2: Invalidate cache data on overwrite/append
## What changes were proposed in this pull request?

File source V2 currently incorrectly continues to use cached data even if the underlying data is overwritten.
We should follow https://github.com/apache/spark/pull/13566 and fix it by invalidating and refreshes all the cached data (and the associated metadata) for any Dataframe that contains the given data source path.

## How was this patch tested?

Unit test

Closes #24318 from gengliangwang/invalidCache.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-09 09:25:37 -07:00
francis0407 601fac2cb3 [SPARK-27411][SQL] DataSourceV2Strategy should not eliminate subquery
## What changes were proposed in this pull request?

In DataSourceV2Strategy, it seems we eliminate the subqueries by mistake after normalizing filters.
We have a sql with a scalar subquery:

``` scala
val plan = spark.sql("select * from t2 where t2a > (select max(t1a) from t1)")
plan.explain(true)
```

And we get the log info of DataSourceV2Strategy:
```
Pushing operators to csv:examples/src/main/resources/t2.txt
Pushed Filters:
Post-Scan Filters: isnotnull(t2a#30)
Output: t2a#30, t2b#31
```

The `Post-Scan Filters` should contain the scalar subquery, but we eliminate it by mistake.
```
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('t2a > scalar-subquery#56 [])
   :  +- 'Project [unresolvedalias('max('t1a), None)]
   :     +- 'UnresolvedRelation `t1`
   +- 'UnresolvedRelation `t2`

== Analyzed Logical Plan ==
t2a: string, t2b: string
Project [t2a#30, t2b#31]
+- Filter (t2a#30 > scalar-subquery#56 [])
   :  +- Aggregate [max(t1a#13) AS max(t1a)#63]
   :     +- SubqueryAlias `t1`
   :        +- RelationV2[t1a#13, t1b#14] csv:examples/src/main/resources/t1.txt
   +- SubqueryAlias `t2`
      +- RelationV2[t2a#30, t2b#31] csv:examples/src/main/resources/t2.txt

== Optimized Logical Plan ==
Filter (isnotnull(t2a#30) && (t2a#30 > scalar-subquery#56 []))
:  +- Aggregate [max(t1a#13) AS max(t1a)#63]
:     +- Project [t1a#13]
:        +- RelationV2[t1a#13, t1b#14] csv:examples/src/main/resources/t1.txt
+- RelationV2[t2a#30, t2b#31] csv:examples/src/main/resources/t2.txt

== Physical Plan ==
*(1) Project [t2a#30, t2b#31]
+- *(1) Filter isnotnull(t2a#30)
   +- *(1) BatchScan[t2a#30, t2b#31] class org.apache.spark.sql.execution.datasources.v2.csv.CSVScan
```
## How was this patch tested?

ut

Closes #24321 from francis0407/SPARK-27411.

Authored-by: francis0407 <hanmingcong123@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 21:45:46 +08:00
mingbo_pb 3e4cfe9dbc [SPARK-27406][SQL] UnsafeArrayData serialization breaks when two machines have different Oops size
## What changes were proposed in this pull request?
ApproxCountDistinctForIntervals holds the UnsafeArrayData data to initialize endpoints. When the UnsafeArrayData is serialized with Java serialization, the BYTE_ARRAY_OFFSET in memory can change if two machines have different pointer width (Oops in JVM).

This PR fixes this issue by using the same way in https://github.com/apache/spark/pull/9030

## How was this patch tested?
Manual test has been done in our tpcds environment and regarding unit test case has been added as well

Closes #24317 from pengbo/SPARK-27406.

Authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 15:41:42 +08:00
Hyukjin Kwon f16dfb9129 [SPARK-27328][SQL] Add 'deprecated' in ExpressionDescription for extended usage and SQL doc
## What changes were proposed in this pull request?

This PR proposes to two things:

1. Add `deprecated` field to `ExpressionDescription` so that it can be shown in our SQL function documentation (https://spark.apache.org/docs/latest/api/sql/), and it can be shown via `DESCRIBE FUNCTION EXTENDED`.

2. While I am here, add some more restrictions for `note()` and `since()`. Looks some documentations are broken due to malformed `note`:

    ![Screen Shot 2019-03-31 at 3 00 53 PM](https://user-images.githubusercontent.com/6477701/55285518-a3e88500-53c8-11e9-9e99-41d857794fbe.png)

    It should start with 4 spaces and end with a newline. I added some asserts, and fixed the instances together while I am here. This is technically a breaking change but I think it's too trivial to note somewhere (and we're in Spark 3.0.0).

This PR adds `deprecated` property into `from_utc_timestamp` and `to_utc_timestamp` (it's deprecated as of #24195) as examples of using this field.

Now it shows the deprecation information as below:

- **SQL documentation is shown as below:**

    ![Screen Shot 2019-03-31 at 3 07 31 PM](https://user-images.githubusercontent.com/6477701/55285537-2113fa00-53c9-11e9-9932-f5693a03332d.png)

- **`DESCRIBE FUNCTION EXTENDED from_utc_timestamp;`**:

    ```
    Function: from_utc_timestamp
    Class: org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp
    Usage: from_utc_timestamp(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.
    Extended Usage:
        Examples:
          > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul');
           2016-08-31 09:00:00

        Since: 1.5.0

        Deprecated:
          Deprecated since 3.0.0. See SPARK-25496.

    ```

## How was this patch tested?

Manually tested via:

- For documentation verification:

    ```
    $ cd sql
    $ sh create-docs.sh
    ```

- For checking description:

    ```
    $ ./bin/spark-sql
    ```
    ```
    spark-sql> DESCRIBE FUNCTION EXTENDED from_utc_timestamp;
    spark-sql> DESCRIBE FUNCTION EXTENDED to_utc_timestamp;
    ```

Closes #24259 from HyukjinKwon/SPARK-27328.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 13:49:42 +08:00
Wenchen Fan 051336d9dd [SPARK-25496][SQL][FOLLOWUP] avoid using to_utc_timestamp
## What changes were proposed in this pull request?

in https://github.com/apache/spark/pull/24195 , we deprecate `from/to_utc_timestamp`.

This PR removes unnecessary use of `to_utc_timestamp` in the test.

## How was this patch tested?

test only PR

Closes #24319 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 10:13:38 +08:00
Gengliang Wang d50603a37c [SPARK-27271][SQL] Migrate Text to File Data Source V2
## What changes were proposed in this pull request?

Migrate Text source to File Data Source V2

## How was this patch tested?

Unit test

Closes #24207 from gengliangwang/textV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 10:15:22 -07:00
Maxim Gekk 00241733a6 [SPARK-27405][SQL][TEST] Restrict the range of generated random timestamps
## What changes were proposed in this pull request?

In the PR, I propose to restrict the range of random timestamp literals generated in `LiteralGenerator. timestampLiteralGen`. The generator creates instances of `java.sql.Timestamp` by passing milliseconds since epoch as `Long` type. Converting the milliseconds to microseconds can cause arithmetic overflow of Long type because Catalyst's Timestamp type stores microseconds since epoch in `Long` type internally as well. Proposed interval of random milliseconds is `[Long.MinValue / 1000, Long.MaxValue / 1000]`.

For example, generated timestamp `new java.sql.Timestamp(-3948373668011580000)` causes `Long` overflow at the method:
```scala
  def fromJavaTimestamp(t: Timestamp): SQLTimestamp = {
  ...
      MILLISECONDS.toMicros(t.getTime()) + NANOSECONDS.toMicros(t.getNanos()) % NANOS_PER_MICROS
  ...
  }
```
because `t.getTime()` returns `-3948373668011580000` which is multiplied by `1000` at `MILLISECONDS.toMicros`, and the result `-3948373668011580000000` is less than `Long.MinValue`.

## How was this patch tested?

By `DateExpressionsSuite` in the PR https://github.com/apache/spark/pull/24311

Closes #24316 from MaxGekk/random-timestamps-gen.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 09:53:00 -07:00
Yuming Wang 33f3c48cac [SPARK-27176][SQL] Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4
## What changes were proposed in this pull request?

This PR mainly contains:
1. Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4.
2. Resolve compatibility issues between Hive 1.2.1 and Hive 2.3.4 in the `sql/hive` module.

## How was this patch tested?
jenkins test hadoop-2.7
manual test hadoop-3:
```shell
build/sbt clean package -Phadoop-3.2 -Phive
export SPARK_PREPEND_CLASSES=true

# rm -rf metastore_db

cat <<EOF > test_hadoop3.scala
spark.range(10).write.saveAsTable("test_hadoop3")
spark.table("test_hadoop3").show
EOF

bin/spark-shell --conf spark.hadoop.hive.metastore.schema.verification=false --conf spark.hadoop.datanucleus.schema.autoCreateAll=true -i test_hadoop3.scala
```

Closes #23788 from wangyum/SPARK-23710-hadoop3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-08 08:42:21 -07:00
Michael Allman 215609def2 [SPARK-25407][SQL] Allow nested access for non-existent field for Parquet file when nested pruning is enabled
## What changes were proposed in this pull request?

As part of schema clipping in `ParquetReadSupport.scala`, we add fields in the Catalyst requested schema which are missing from the Parquet file schema to the Parquet clipped schema. However, nested schema pruning requires we ignore unrequested field data when reading from a Parquet file. Therefore we pass two schema to `ParquetRecordMaterializer`: the schema of the file data we want to read and the schema of the rows we want to return. The reader is responsible for reconciling the differences between the two.

Aside from checking whether schema pruning is enabled, there is an additional complication to constructing the Parquet requested schema. The manner in which Spark's two Parquet readers reconcile the differences between the Parquet requested schema and the Catalyst requested schema differ. Spark's vectorized reader does not (currently) support reading Parquet files with complex types in their schema. Further, it assumes that the Parquet requested schema includes all fields requested in the Catalyst requested schema. It includes logic in its read path to skip fields in the Parquet requested schema which are not present in the file.

Spark's parquet-mr based reader supports reading Parquet files of any kind of complex schema, and it supports nested schema pruning as well. Unlike the vectorized reader, the parquet-mr reader requires that the Parquet requested schema include only those fields present in the underlying Parquet file's schema. Therefore, in the case where we use the parquet-mr reader we intersect the Parquet clipped schema with the Parquet file's schema to construct the Parquet requested schema that's set in the `ReadContext`.

_Additional description (by HyukjinKwon):_

Let's suppose that we have a Parquet schema as below:

```
message spark_schema {
  required int32 id;
  optional group name {
    optional binary first (UTF8);
    optional binary last (UTF8);
  }
  optional binary address (UTF8);
}
```

Currently, the clipped schema as follows:

```
message spark_schema {
  optional group name {
    optional binary middle (UTF8);
  }
  optional binary address (UTF8);
}
```

Parquet MR does not support access to the nested non-existent field (`name.middle`).

To workaround this, this PR removes `name.middle` request at all to Parquet reader as below:

```
Parquet requested schema:
message spark_schema {
  optional binary address (UTF8);
}
```

and produces the record (`name.middle`) properly as the requested Catalyst schema.

```
root
-- name: struct (nullable = true)
    |-- middle: string (nullable = true)
-- address: string (nullable = true)
```

I think technically this is what Parquet library should support since Parquet library made a design decision to produce `null` for non-existent fields IIRC. This PR targets to work around it.

## How was this patch tested?

A previously ignored test case which exercises the failure scenario this PR addresses has been enabled.

This closes #22880

Closes #24307 from dongjoon-hyun/SPARK-25407.

Lead-authored-by: Michael Allman <msa@allman.ms>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-08 22:26:02 +09:00
Gengliang Wang 02e9f93309 [SPARK-27384][SQL] File source V2: Prune unnecessary partition columns
## What changes were proposed in this pull request?

When scanning file sources, we can prune unnecessary partition columns on constructing input partitions, so that:
1. Reduce the data transformation from Driver to Executors
2. Make it easier to implement columnar batch readers, since the partition columns are already pruned.
## How was this patch tested?

Existing unit tests.

Closes #24296 from gengliangwang/prunePartitionValue.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-08 15:14:02 +08:00
Yuming Wang 017919b636 [SPARK-27383][SQL][TEST] Avoid using hard-coded jar names in Hive tests
## What changes were proposed in this pull request?

This pr avoid using hard-coded jar names(`hive-contrib-0.13.1.jar` and `hive-hcatalog-core-0.13.1.jar`) in Hive tests. This change makes it easy to change when upgrading the built-in Hive to 2.3.4.

## How was this patch tested?

Existing test

Closes #24294 from wangyum/SPARK-27383.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 18:06:52 -05:00
cxzl25 6450c5948a [SPARK-26992][STS] Fix STS scheduler pool correct delivery
## What changes were proposed in this pull request?

The user sets the value of spark.sql.thriftserver.scheduler.pool.
Spark thrift server saves this value in the LocalProperty of threadlocal type, but does not clean up after running, causing other sessions to run in the previously set pool name.

## How was this patch tested?

manual tests

Closes #23895 from cxzl25/thrift_server_scheduler_pool_pollute.

Lead-authored-by: cxzl25 <cxzl25@users.noreply.github.com>
Co-authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 17:14:29 -05:00
Jose Torres 4a5768b2a2 [SPARK-27391][SS] Don't initialize a lazy val in ContinuousExecution job.
## What changes were proposed in this pull request?

Fix a potential deadlock in ContinuousExecution by not initializing the toRDD lazy val.

Closes #24301 from jose-torres/deadlock.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Jose Torres <torres.joseph.f+github@gmail.com>
2019-04-05 12:56:36 -07:00
gengjiaan 979bb905b7 [SPARK-26936][SQL] Fix bug of insert overwrite local dir can not create temporary path in local staging directory
## What changes were proposed in this pull request?
Th environment of my cluster as follows:
```
OS:Linux version 2.6.32-220.7.1.el6.x86_64 (mockbuildc6b18n3.bsys.dev.centos.org) (gcc version 4.4.6 20110731 (Red Hat 4.4.6-3) (GCC) ) #1 SMP Wed Mar 7 00:52:02 GMT 2012
Hadoop: 2.7.2
Spark: 2.3.0 or 3.0.0(master branch)
Hive: 1.2.1
```

My spark run on deploy mode yarn-client.

If I execute the SQL `insert overwrite local directory '/home/test/call_center/' select * from call_center`, a HiveException will appear as follows:
`Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: Mkdirs failed to create file:/home/xitong/hive/stagingdir_hive_2019-02-19_17-31-00_678_1816816774691551856-1/-ext-10000/_temporary/0/_temporary/attempt_20190219173233_0002_m_000000_3 (exists=false, cwd=file:/data10/yarn/nm-local-dir/usercache/xitong/appcache/application_1543893582405_6126857/container_e124_1543893582405_6126857_01_000011)
at org.apache.hadoop.hive.ql.io.HiveFileFormatUtils.getHiveRecordWriter(HiveFileFormatUtils.java:249)`
Current spark sql generate a local temporary path in local staging directory.The schema of local temporary path start with `file`, so the HiveException appears.
This PR change the local temporary path to HDFS temporary path, and use DistributedFileSystem instance copy the data from HDFS temporary path to local directory.
If Spark run on local deploy mode, 'insert overwrite local directory' works fine.
## How was this patch tested?

UT cannot support yarn-client mode.The test is in my product environment.

Closes #23841 from beliefer/fix-bug-of-insert-overwrite-local-dir.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 14:02:46 -05:00
Dongjoon Hyun 982c4c8e3c [SPARK-27390][CORE][SQL][TEST] Fix package name mismatch
## What changes were proposed in this pull request?

This PR aims to clean up package name mismatches.

## How was this patch tested?

Pass the Jenkins.

Closes #24300 from dongjoon-hyun/SPARK-27390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-05 11:50:37 -07:00
gatorsmile 5678e687c6 [SPARK-27393][SQL] Show ReusedSubquery in the plan when the subquery is reused
## What changes were proposed in this pull request?
With this change, we can easily identify the plan difference when subquery is reused.

When the reuse is enabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery240 + ReusedSubquery Subquery subquery240) AS (scalarsubquery() + scalarsubquery())#253]
   :  :- Subquery subquery240
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#250])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#256, count#257L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- ReusedSubquery Subquery subquery240
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

When the reuse is disabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery286 + Subquery subquery287) AS (scalarsubquery() + scalarsubquery())#299]
   :  :- Subquery subquery286
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#296])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#302, count#303L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- Subquery subquery287
   :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#298])
   :        +- Exchange SinglePartition
   :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#306, count#307L])
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :                 +- Scan[obj#12]
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

## How was this patch tested?
Modified the existing test.

Closes #24258 from gatorsmile/followupSPARK-27279.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-05 08:31:41 -07:00
Gengliang Wang 568db94e0c [SPARK-27356][SQL] File source V2: Fix the case that data columns overlap with partition schema
## What changes were proposed in this pull request?

In the current file source V2 framework, the schema of `FileScan` is not returned correctly if there are overlap columns between `dataSchema` and `partitionSchema`. The actual schema should be
`dataSchema - overlapSchema + partitionSchema`, which might have different column order from the pushed down `requiredSchema` in `SupportsPushDownRequiredColumns.pruneColumns`.

For example, if the data schema is `[a: String, b: String, c: String]` and the partition schema is `[b: Int, d: Int]`, the result schema is `[a: String, b: Int, c: String, d: Int]` in current `FileTable` and `HadoopFsRelation`. while the actual scan schema is `[a: String, c: String, b: Int, d: Int]` in `FileScan`.

To fix the corner case, this PR proposes that the output schema of `FileTable` should be `dataSchema - overlapSchema + partitionSchema`, so that the column order is consistent with `FileScan`.
Putting all the partition columns to the end of table schema is more reasonable.

## How was this patch tested?

Unit test.

Closes #24284 from gengliangwang/FixReadSchema.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-05 13:34:46 +08:00
Aayushmaan Jain 04e53d2e3c [SPAR-27342][SQL] Optimize Limit 0 queries
## What changes were proposed in this pull request?
With this change, unnecessary file scans are avoided in case of Limit 0 queries.

I added a case (rule) to `PropagateEmptyRelation` to replace `GlobalLimit 0` and `LocalLimit 0` nodes with an empty `LocalRelation`. This prunes the subtree under the Limit 0 node and further allows other rules of `PropagateEmptyRelation` to optimize the Logical Plan - while remaining semantically consistent with the Limit 0 query.

For instance:
**Query:**
`SELECT * FROM table1 INNER JOIN (SELECT * FROM table2 LIMIT 0) AS table2 ON table1.id = table2.id`

**Optimized Plan without fix:**
```
Join Inner, (id#79 = id#87)
:- Filter isnotnull(id#79)
:  +- Relation[id#79,num1#80] parquet
+- Filter isnotnull(id#87)
   +- GlobalLimit 0
      +- LocalLimit 0
         +- Relation[id#87,num2#88] parquet
```

**Optimized Plan with fix:**
`LocalRelation <empty>, [id#75, num1#76, id#77, num2#78]`

## How was this patch tested?
Added unit tests to verify Limit 0 optimization for:
- Simple query containing Limit 0
- Inner Join, Left Outer Join, Right Outer Join, Full Outer Join queries containing Limit 0 as one of their children
- Nested Inner Joins between 3 tables with one of them having a Limit 0 clause.
- Intersect query wherein one of the subqueries was a Limit 0 query.

Closes #24271 from aayushmaanjain/optimize-limit0.

Authored-by: Aayushmaan Jain <aayushmaan.jain42@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-04 21:19:40 -07:00
Ruben Fiszel 0e44a51f2e [SPARK-24345][SQL] Improve ParseError stop location when offending symbol is a token
In the case where the offending symbol is a CommonToken, this PR increases the accuracy of the start and stop origin by leveraging the start and stop index information from CommonToken.

Closes #21334 from rubenfiszel/patch-1.

Lead-authored-by: Ruben Fiszel <rubenfiszel@gmail.com>
Co-authored-by: rubenfiszel <rfiszel@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-04 18:20:34 -05:00
Dongjoon Hyun 938d954375 [SPARK-27382][SQL][TEST] Update Spark 2.4.x testing in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

Since Apache Spark 2.4.1 vote passed and is distributed into mirrors, we need to test 2.4.1. This should land on both `master` and `branch-2.4`.

## How was this patch tested?

Pass the Jenkins.

Closes #24292 from dongjoon-hyun/SPARK-27382.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-04 13:49:56 -07:00
Wenchen Fan f7bd1ab586 [SPARK-26811][SQL][FOLLOWUP] some more document fixes
## What changes were proposed in this pull request?

while working on https://github.com/apache/spark/pull/24129, I realized that I missed some document fixes in https://github.com/apache/spark/pull/24285. This PR covers all of them.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #24295 from cloud-fan/doc.
2019-04-05 01:07:08 +08:00
Yuming Wang 1d95dea307 [SPARK-27349][SQL] Dealing with TimeVars removed in Hive 2.x
## What changes were proposed in this pull request?
`hive.stats.jdbc.timeout` and `hive.stats.retries.wait` were removed by [HIVE-12164](https://issues.apache.org/jira/browse/HIVE-12164).
This pr to deal with this change.

## How was this patch tested?

unit tests

Closes #24277 from wangyum/SPARK-27349.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-03 22:52:37 -07:00
Wenchen Fan 5c50f68253 [SPARK-26811][SQL][FOLLOWUP] fix some documentation
## What changes were proposed in this pull request?

It's a followup of https://github.com/apache/spark/pull/24012 , to fix 2 documentation:
1. `SupportsRead` and `SupportsWrite` are not internal anymore. They are public interfaces now.
2. `Scan` should link the `BATCH_READ` instead of hardcoding it.

## How was this patch tested?
N/A

Closes #24285 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 10:31:27 +08:00
Dongjoon Hyun b51763612a Revert "[SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not"
This reverts commit 5888b15d9c.
2019-04-03 09:41:13 -07:00
Wenchen Fan ffb362a705 [SPARK-19712][SQL][FOLLOW-UP] reduce code duplication
## What changes were proposed in this pull request?

abstract some common code into a method.

## How was this patch tested?

existing tests

Closes #24281 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 00:37:57 +08:00
Liang-Chi Hsieh d04a7371da [MINOR][DOC][SQL] Remove out-of-date doc about ORC in DataFrameReader and Writer
## What changes were proposed in this pull request?

According to current status, `orc` is available even Hive support isn't enabled. This is a minor doc change to reflect it.

## How was this patch tested?

Doc only change.

Closes #24280 from viirya/fix-orc-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-03 09:11:09 -07:00
Maxim Gekk 1bc672366d [SPARK-27344][SQL][TEST] Support the LocalDate and Instant classes in Java Bean encoders
## What changes were proposed in this pull request?

- Added new test for Java Bean encoder of the classes: `java.time.LocalDate` and `java.time.Instant`.
- Updated comment for `Encoders.bean`
- New Row getters: `getLocalDate` and `getInstant`
- Extended `inferDataType` to infer types for `java.time.LocalDate` -> `DateType` and `java.time.Instant` -> `TimestampType`.

## How was this patch tested?

By `JavaBeanDeserializationSuite`

Closes #24273 from MaxGekk/bean-instant-localdate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 17:45:59 +08:00
Dilip Biswal 3286bff942 [SPARK-27255][SQL] Report error when illegal expressions are hosted by a plan operator.
## What changes were proposed in this pull request?
In the PR, we raise an AnalysisError when we detect the presense of aggregate expressions in where clause. Here is the problem description from the JIRA.

Aggregate functions should not be allowed in WHERE clause. But Spark SQL throws an exception when generating codes. It is supposed to throw an exception during parsing or analyzing.

Here is an example:
```
val df = spark.sql("select * from t where sum(ta) > 0")
df.explain(true)
df.show()
```
Resulting exception:
```
Exception in thread "main" java.lang.UnsupportedOperationException: Cannot generate code for expression: sum(cast(input[0, int, false] as bigint))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:291)
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:290)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:138)
	at scala.Option.getOrElse(Option.scala:138)
```
Checked the behaviour of other database and all of them return an exception:
**Postgress**
```
select * from foo where max(c1) > 0;
Error
ERROR: aggregate functions are not allowed in WHERE Position: 25
```
**DB2**
```
db2 => select * from foo where max(c1) > 0;
SQL0120N  Invalid use of an aggregate function or OLAP function.
```
**Oracle**
```
select * from foo where max(c1) > 0;
ORA-00934: group function is not allowed here
```
**MySql**
```
select * from foo where max(c1) > 0;
Invalid use of group function
```

**Update**
This PR has been enhanced to report error when expressions such as Aggregate, Window, Generate are hosted by operators where they are invalid.
## How was this patch tested?
Added tests in AnalysisErrorSuite and group-by.sql

Closes #24209 from dilipbiswal/SPARK-27255.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 13:05:06 +08:00
Maxim Gekk 1d20d13149 [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 10:55:56 +08:00
Dilip Biswal b8b5acdd41 [SPARK-19712][SQL][FOLLOW-UP] Don't do partial pushdown when pushing down LeftAnti joins below Aggregate or Window operators.
## What changes were proposed in this pull request?
After [23750](https://github.com/apache/spark/pull/23750), we may pushdown left anti joins below aggregate and window operators with a partial join condition. This is not correct and was pointed out by hvanhovell and cloud-fan [here](https://github.com/apache/spark/pull/23750#discussion_r270017097). This pr addresses their comments.
## How was this patch tested?
Added two new tests to verify the behaviour.

Closes #24253 from dilipbiswal/SPARK-19712-followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 09:56:27 +08:00
Hyukjin Kwon d7dd59a6b4 [SPARK-26224][SQL][PYTHON][R][FOLLOW-UP] Add notes about many projects in withColumn at SparkR and PySpark as well
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23285. This PR adds the notes into PySpark and SparkR documentation as well.

While I am here, I revised the doc a bit to make it sound a bit more neutral

## How was this patch tested?

Manually built the doc and verified.

Closes #24272 from HyukjinKwon/SPARK-26224.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:30:24 +09:00
Hyukjin Kwon 949d712839 [SPARK-27346][SQL] Loosen the newline assert condition on 'examples' field in ExpressionInfo
## What changes were proposed in this pull request?

I haven't tested by myself on Windows and I am not 100% sure if this is going to cause an actual problem.

However, this one line:

827383a97c/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java (L82)

made me to investigate a lot today.

Given my speculation, if Spark is built in Linux and it's executed on Windows, it looks possible for multiline strings, like,

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala (L146-L150)

to throw an exception because the newline in the binary is `\n` but `System.lineSeparator` returns `\r\n`.

I think this is not yet found because this particular codes are not released yet (see SPARK-26426).

Looks just better to loosen the condition and forget about this stuff.

This should be backported into branch-2.4 as well.

## How was this patch tested?

N/A

Closes #24274 from HyukjinKwon/SPARK-27346.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:27:41 +09:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
Dongjoon Hyun d575a453db Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit c5e83ab92c.
2019-04-02 01:05:54 -07:00
Marco Gaido 0b150f833c [SPARK-26224][SQL] Advice the user when creating many project on subsequent calls to withColumn
## What changes were proposed in this pull request?

We have seen many cases when users make several subsequent calls to `withColumn` on a Dataset. This leads now to the generation of a lot of `Project` nodes on the top of the plan, with serious problem which can lead also to `StackOverflowException`s.

The PR improves the doc of `withColumn`, in order to advise the user to avoid this pattern and do something different, ie. a single select with all the column he/she needs.

## How was this patch tested?

NA

Closes #23285 from mgaido91/SPARK-26224.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-02 14:12:47 +09:00
Maxim Gekk c5e83ab92c [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-02 10:20:06 +08:00
Liang-Chi Hsieh eaf008ad0e [SPARK-27329][SQL] Pruning nested field in map of map key and value from object serializers
## What changes were proposed in this pull request?

If object serializer has map of map key/value, pruning nested field should work.

Previously object serializer pruner don't recursively prunes nested fields if it is deeply located in map key or value. This patch proposed to address it by slightly factoring the pruning logic.

## How was this patch tested?

Added tests.

Closes #24260 from viirya/SPARK-27329.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 13:53:55 -07:00
Marco Gaido 5888b15d9c [SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not
## What changes were proposed in this pull request?

When `GetMapValue` contains a foldable Map and a non-foldable key, `SimplifyExtractValueOps` fails to optimize it transforming it into case when statements.
The PR adds a case for covering this situation too.

## How was this patch tested?

added UT

Closes #24223 from mgaido91/SPARK-27278.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 09:09:06 -07:00
Maxim Gekk d332958109 [SPARK-27325][SQL] Add implicit encoders for LocalDate and Instant
## What changes were proposed in this pull request?

Added implicit encoders for the `java.time.LocalDate` and `java.time.Instant` classes. This allows creation of datasets from instances of the types.

## How was this patch tested?

Added new tests to `JavaDatasetSuite` and `DatasetSuite`.

Closes #24249 from MaxGekk/instant-localdate-encoders.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 23:02:48 +08:00
Marco Gaido 8012f55a9b [SPARK-26812][SQL] Report correct nullability for complex datatypes in Union
## What changes were proposed in this pull request?

When there is a `Union`, the reported output datatypes are the ones of the first plan and the nullability is updated according to all the plans. For complex types, though, the nullability of their elements is not updated using the types from the other plans. This means that the nullability of the inner elements is the one of the first plan. If this is not compatible with the one of other plans, errors can happen (as reported in the JIRA).

The PR proposes to update the nullability of the inner elements of complex datatypes according to most permissive value of all the plans.

## How was this patch tested?

added UT

Closes #23726 from mgaido91/SPARK-26812.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 22:22:10 +08:00
chakravarthiT fc9aad0957 [SPARK-27253][SQL] Prioritizes parent session's SQLConf over SparkConf when cloning a session
## What changes were proposed in this pull request?

Cloned session should prioritize `SQLConf` from parent's over `SparkConf`. Currently, when cloning a session, the child session has configuration set in `SparkConf` even the same properties are set to its parent `SQLConf`.

Currently, when a Spark session is cloned, `mergeSparkConf` in `BaseSessionStateBuilder`'s `conf` overwrites  `SQLConf` values as set in `SparkConf`.

This PR proposes to call `mergeSparkConf` only when the parent session is empty.

See below codes to read.

1. Parent's `sessionState`

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L268)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L157-L161)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L88-L90)

2. Child `sessionState`

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L269)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L155)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala (L102)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala (L74)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L305)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L283)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L292)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L88-L90)

## How was this patch tested?
Added UT and with existing Unit Tests.

Closes #24189 from chakravarthiT/CloneDiscardsConf.

Authored-by: chakravarthiT <tcchakra@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-01 09:33:18 +09:00
Takeshi Yamamuro 885aab40a2 [SPARK-27266][SQL] Support ANALYZE TABLE to collect tables stats for cached catalog views
## What changes were proposed in this pull request?
The current master doesn't support ANALYZE TABLE to collect tables stats for catalog views even if they are cached as follows;

```scala
scala> sql(s"CREATE VIEW v AS SELECT 1 c")
scala> sql(s"CACHE LAZY TABLE v")
scala> sql(s"ANALYZE TABLE v COMPUTE STATISTICS")
org.apache.spark.sql.AnalysisException: ANALYZE TABLE is not supported on views.;
...
```

Since SPARK-25196 has supported to an ANALYZE command to collect column statistics for cached catalog view, we could support table stats, too.

## How was this patch tested?
Added tests in `StatisticsCollectionSuite` and `InMemoryColumnarQuerySuite`.

Closes #24200 from maropu/SPARK-27266.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-31 17:24:21 -07:00
Maxim Gekk 6115a5e1a0 [SPARK-27327][SQL] New JSON benchmarks: functions, Dataset[String]
## What changes were proposed in this pull request?

Added new benchmarks for:
1. JSON functions: `from_json`, `json_tuple` and `get_json_object`
2. Parsing `Dataset[String]` with JSON records
3. Comparing just splitting input text by lines with schema inferring, per-line parsing when encoding is set and not set.

Also existing benchmarks were refactored to use the `NoOp` datasource to eliminate overhead of triggers like `.filter((_: Row) => true).count()`.

## How was this patch tested?

By running `JSONBenchmark` locally.

Closes #24252 from MaxGekk/json-benchmark-func.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-01 08:33:16 +09:00
Gengliang Wang 5dab5f651f [SPARK-27326][SQL] Fall back all v2 file sources in InsertIntoTable to V1 FileFormat
## What changes were proposed in this pull request?

In the first PR for file source V2, there was a rule for falling back Orc V2 table to OrcFileFormat: https://github.com/apache/spark/pull/23383/files#diff-57e8244b6964e4f84345357a188421d5R34

As we are migrating more file sources to data source V2, we should make the rule more generic. This PR proposes to:
1. Rename the rule `FallbackOrcDataSourceV2 ` to `FallBackFileSourceV2`.The name is more generic. And we use "fall back" as verb, while "fallback" is noun.
2. Rename the method `fallBackFileFormat` in `FileDataSourceV2` to `fallbackFileFormat`. Here we should use "fallback" as noun.
3. Add new method `fallbackFileFormat` in `FileTable`. This is for falling back to V1 in rule `FallbackOrcDataSourceV2 `.

## How was this patch tested?

Existing Unit tests.

Closes #24251 from gengliangwang/fallbackV1Rule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-30 14:38:26 -07:00
10129659 144b35fe3a [SPARK-27320][SQL] Replacing index with iterator to traverse the expressions list in AggregationIterator, which make it simpler
## What changes were proposed in this pull request?
In AggregationIterator's loop function, we access the expressions by `expressions(i)`, the type of `expressions` is `::`, a subtype of list.

```
while (i < expressionsLength) {
      val func = expressions(i).aggregateFunction
```

This PR replacing  index with iterator to access the expressions list, which make it simpler.

## How was this patch tested?
Existing tests.

Closes #24238 from eatoncys/array.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 02:27:12 -05:00
Takuya UESHIN f176dd3f28 [SPARK-27314][SQL] Deduplicate exprIds for Union.
## What changes were proposed in this pull request?

We have been having a potential problem with `Union` when the children have the same expression id in their outputs, which happens when self-union.

## How was this patch tested?

Modified some tests to adjust plan changes.

Closes #24236 from ueshin/issues/SPARK-27314/dedup_union.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-29 14:05:38 -07:00
Maxim Gekk 61561c1c2d [SPARK-27252][SQL][FOLLOWUP] Calculate min and max days independently from time zone in ComputeCurrentTimeSuite
## What changes were proposed in this pull request?

This fixes the `analyzer should replace current_date with literals` test in `ComputeCurrentTimeSuite` by making calculation of `min` and `max` days independent from time zone.

## How was this patch tested?

by `ComputeCurrentTimeSuite`.

Closes #24240 from MaxGekk/current-date-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-29 14:28:36 -05:00
Maxim Gekk 06abd06112 [SPARK-27252][SQL] Make current_date() independent from time zones
## What changes were proposed in this pull request?

This makes the `CurrentDate` expression and `current_date` function independent from time zone settings. New result is number of days since epoch in `UTC` time zone. Previously, Spark shifted the current date (in `UTC` time zone) according the session time zone which violets definition of `DateType` - number of days since epoch (which is an absolute point in time, midnight of Jan 1 1970 in UTC time).

The changes makes `CurrentDate` consistent to `CurrentTimestamp` which is independent from time zone too.

## How was this patch tested?

The changes were tested by existing test suites like `DateExpressionsSuite`.

Closes #24185 from MaxGekk/current-date.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-28 18:44:08 -07:00
Xianyang Liu 50cded590f [MINOR] Move java file to java directory
## What changes were proposed in this pull request?

move
```scala
org.apache.spark.sql.execution.streaming.BaseStreamingSource
org.apache.spark.sql.execution.streaming.BaseStreamingSink
```
to java directory

## How was this patch tested?

Existing UT.

Closes #24222 from ConeyLiu/move-scala-to-java.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-28 12:11:00 -05:00
zhoukang 43bf4ae641 [SPARK-26914][SQL] Fix scheduler pool may be unpredictable when we only want to use default pool and do not set spark.scheduler.pool for the session
## What changes were proposed in this pull request?

When using fair scheduler mode for thrift server, we may have unpredictable result.
```
val pool = sessionToActivePool.get(parentSession.getSessionHandle)
if (pool != null) {
   sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)
}
```
The cause is we use thread pool to execute queries for thriftserver, and when we call setLocalProperty we may have unpredictab behavior.

```
/**
   * Set a local property that affects jobs submitted from this thread, such as the Spark fair
   * scheduler pool. User-defined properties may also be set here. These properties are propagated
   * through to worker tasks and can be accessed there via
   * [[org.apache.spark.TaskContext#getLocalProperty]].
   *
   * These properties are inherited by child threads spawned from this thread. This
   * may have unexpected consequences when working with thread pools. The standard java
   * implementation of thread pools have worker threads spawn other worker threads.
   * As a result, local properties may propagate unpredictably.
   */
  def setLocalProperty(key: String, value: String) {
    if (value == null) {
      localProperties.get.remove(key)
    } else {
      localProperties.get.setProperty(key, value)
    }
  }
```

I post an example on https://jira.apache.org/jira/browse/SPARK-26914 .

## How was this patch tested?
UT

Closes #23826 from caneGuy/zhoukang/fix-scheduler-error.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-28 09:24:16 -05:00
Gengliang Wang 49b0411549 [SPARK-27291][SQL] PartitioningAwareFileIndex: Filter out empty files on listing files
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23130, all empty files are excluded from target file splits in `FileSourceScanExec`.
In File source V2, we should keep the same behavior.

This PR suggests to filter out empty files on listing files in `PartitioningAwareFileIndex` so that the upper level doesn't need to handle them.
## How was this patch tested?

Unit test

Closes #24227 from gengliangwang/ignoreEmptyFile.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-27 10:08:38 -07:00
Daoyuan Wang f1fe805bed [SPARK-27279][SQL] Reuse subquery should compare child plan of SubqueryExec
## What changes were proposed in this pull request?

For now, `ReuseSubquery` in Spark compares two subqueries at `SubqueryExec` level, which invalidates the `ReuseSubquery` rule. This pull request fixes this, and add a configuration key for subquery reuse exclusively.

## How was this patch tested?

add a unit test.

Closes #24214 from adrian-wang/reuse.

Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-27 08:45:22 -07:00
Takeshi Yamamuro 956b52b167 [SPARK-26771][SQL][FOLLOWUP] Make all the uncache operations non-blocking by default
## What changes were proposed in this pull request?
To make the blocking behaviour consistent, this pr made catalog table/view `uncacheQuery` non-blocking by default. If this pr merged, all the behaviours in spark are non-blocking by default.

## How was this patch tested?
Pass Jenkins.

Closes #24212 from maropu/SPARK-26771-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 21:01:36 +09:00