## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
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>
## 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>
## 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>
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>
## 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>
## 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>
## 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>
## 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>
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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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#24591Closes#24391 from wangyum/SPARK-27402.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>