## What changes were proposed in this pull request?
If the user changes the shuffle partition number between batches, Streaming aggregation will fail.
Here are some possible cases:
- Change "spark.sql.shuffle.partitions"
- Use "repartition" and change the partition number in codes
- RangePartitioner doesn't generate deterministic partitions. Right now it's safe as we disallow sort before aggregation. Not sure if we will add some operators using RangePartitioner in future.
## How was this patch tested?
- Unit tests
- Manual tests
- forward compatibility tested by using the new `OffsetSeqMetadata` json with Spark v2.1.0
Author: Kunal Khamar <kkhamar@outlook.com>
Closes#17216 from kunalkhamar/num-partitions.
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `from_json` in SQL.
## How was this patch tested?
Added tests in `JsonFunctionsSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17320 from maropu/SPARK-19967.
## What changes were proposed in this pull request?
Allows null values of the pivot column to be included in the pivot values list without throwing NPE
Note this PR was made as an alternative to #17224 but preserves the two phase aggregate operation that is needed for good performance.
## How was this patch tested?
Additional unit test
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#17226 from aray/pivot-null.
## Problem
There are several places where we write out version identifiers in various logs for structured streaming (usually `v1`). However, in the places where we check for this, we throw a confusing error message.
## What changes were proposed in this pull request?
This patch made two major changes:
1. added a `parseVersion(...)` method, and based on this method, fixed the following places the way they did version checking (no other place needed to do this checking):
```
HDFSMetadataLog
- CompactibleFileStreamLog ------------> fixed with this patch
- FileStreamSourceLog ---------------> inherited the fix of `CompactibleFileStreamLog`
- FileStreamSinkLog -----------------> inherited the fix of `CompactibleFileStreamLog`
- OffsetSeqLog ------------------------> fixed with this patch
- anonymous subclass in KafkaSource ---> fixed with this patch
```
2. changed the type of `FileStreamSinkLog.VERSION`, `FileStreamSourceLog.VERSION` etc. from `String` to `Int`, so that we can identify newer versions via `version > 1` instead of `version != "v1"`
- note this didn't break any backwards compatibility -- we are still writing out `"v1"` and reading back `"v1"`
## Exception message with this patch
```
java.lang.IllegalStateException: Failed to read log file /private/var/folders/nn/82rmvkk568sd8p3p8tb33trw0000gn/T/spark-86867b65-0069-4ef1-b0eb-d8bd258ff5b8/0. UnsupportedLogVersion: maximum supported log version is v1, but encountered v99. The log file was produced by a newer version of Spark and cannot be read by this version. Please upgrade.
at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:202)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:78)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:75)
at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:133)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite.withTempDir(OffsetSeqLogSuite.scala:26)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply$mcV$sp(OffsetSeqLogSuite.scala:75)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
```
## How was this patch tested?
unit tests
Author: Liwei Lin <lwlin7@gmail.com>
Closes#17070 from lw-lin/better-msg.
## What changes were proposed in this pull request?
The current master throws `StackOverflowError` in `createDataFrame`/`createDataset` if bean has one's own class in fields;
```
public class SelfClassInFieldBean implements Serializable {
private SelfClassInFieldBean child;
...
}
```
This pr added code to throw `UnsupportedOperationException` in that case as soon as possible.
## How was this patch tested?
Added tests in `JavaDataFrameSuite` and `JavaDatasetSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17188 from maropu/SPARK-19751.
## What changes were proposed in this pull request?
unify a exception erro msg for dropdatabase when the database still have some tables for HiveExternalCatalog and InMemoryCatalog
## How was this patch tested?
N/A
Author: windpiger <songjun@outlook.com>
Closes#17305 from windpiger/unifyErromsg.
## What changes were proposed in this pull request?
Now `InMemoryTableScanExec` simply takes the `outputPartitioning` and `outputOrdering` from the associated `InMemoryRelation`'s `child.outputPartitioning` and `outputOrdering`.
However, `InMemoryTableScanExec` can alias the output attributes. In this case, its `outputPartitioning` and `outputOrdering` are not correct and its parent operators can't correctly determine its data distribution.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17175 from viirya/ensure-no-unnecessary-shuffle.
## What issue does this PR address ?
Jira: https://issues.apache.org/jira/browse/SPARK-13450
In `SortMergeJoinExec`, rows of the right relation having the same value for a join key are buffered in-memory. In case of skew, this causes OOMs (see comments in SPARK-13450 for more details). Heap dump from a failed job confirms this : https://issues.apache.org/jira/secure/attachment/12846382/heap-dump-analysis.png . While its possible to increase the heap size to workaround, Spark should be resilient to such issues as skews can happen arbitrarily.
## Change proposed in this pull request
- Introduces `ExternalAppendOnlyUnsafeRowArray`
- It holds `UnsafeRow`s in-memory upto a certain threshold.
- After the threshold is hit, it switches to `UnsafeExternalSorter` which enables spilling of the rows to disk. It does NOT sort the data.
- Allows iterating the array multiple times. However, any alteration to the array (using `add` or `clear`) will invalidate the existing iterator(s)
- `WindowExec` was already using `UnsafeExternalSorter` to support spilling. Changed it to use the new array
- Changed `SortMergeJoinExec` to use the new array implementation
- NOTE: I have not changed FULL OUTER JOIN to use this new array implementation. Changing that will need more surgery and I will rather put up a separate PR for that once this gets in.
- Changed `CartesianProductExec` to use the new array implementation
#### Note for reviewers
The diff can be divided into 3 parts. My motive behind having all the changes in a single PR was to demonstrate that the API is sane and supports 2 use cases. If reviewing as 3 separate PRs would help, I am happy to make the split.
## How was this patch tested ?
#### Unit testing
- Added unit tests `ExternalAppendOnlyUnsafeRowArray` to validate all its APIs and access patterns
- Added unit test for `SortMergeExec`
- with and without spill for inner join, left outer join, right outer join to confirm that the spill threshold config behaves as expected and output is as expected.
- This PR touches the scanning logic in `SortMergeExec` for _all_ joins (except FULL OUTER JOIN). However, I expect existing test cases to cover that there is no regression in correctness.
- Added unit test for `WindowExec` to check behavior of spilling and correctness of results.
#### Stress testing
- Confirmed that OOM is gone by running against a production job which used to OOM
- Since I cannot share details about prod workload externally, created synthetic data to mimic the issue. Ran before and after the fix to demonstrate the issue and query success with this PR
Generating the synthetic data
```
./bin/spark-shell --driver-memory=6G
import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("DROP TABLE IF EXISTS spark_13450_large_table").collect
hc.sql("DROP TABLE IF EXISTS spark_13450_one_row_table").collect
val df1 = (0 until 1).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df1.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_one_row_table")
val df2 = (0 until 3000000).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df2.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_large_table")
```
Ran this against trunk VS local build with this PR. OOM repros with trunk and with the fix this query runs fine.
```
./bin/spark-shell --driver-java-options="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/spark.driver.heapdump.hprof"
import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("SET spark.sql.autoBroadcastJoinThreshold=1")
hc.sql("SET spark.sql.sortMergeJoinExec.buffer.spill.threshold=10000")
hc.sql("DROP TABLE IF EXISTS spark_13450_result").collect
hc.sql("""
CREATE TABLE spark_13450_result
AS
SELECT
a.i AS a_i, a.j AS a_j, a.str1 AS a_str1, a.str2 AS a_str2,
b.i AS b_i, b.j AS b_j, b.str1 AS b_str1, b.str2 AS b_str2
FROM
spark_13450_one_row_table a
JOIN
spark_13450_large_table b
ON
a.i=b.i AND
a.j=b.j
""")
```
## Performance comparison
### Macro-benchmark
I ran a SMB join query over two real world tables (2 trillion rows (40 TB) and 6 million rows (120 GB)). Note that this dataset does not have skew so no spill happened. I saw improvement in CPU time by 2-4% over version without this PR. This did not add up as I was expected some regression. I think allocating array of capacity of 128 at the start (instead of starting with default size 16) is the sole reason for the perf. gain : https://github.com/tejasapatil/spark/blob/SPARK-13450_smb_buffer_oom/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala#L43 . I could remove that and rerun, but effectively the change will be deployed in this form and I wanted to see the effect of it over large workload.
### Micro-benchmark
Two types of benchmarking can be found in `ExternalAppendOnlyUnsafeRowArrayBenchmark`:
[A] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `ArrayBuffer` when all rows fit in-memory and there is no spill
```
Array with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 7821 / 7941 33.5 29.8 1.0X
ExternalAppendOnlyUnsafeRowArray 8798 / 8819 29.8 33.6 0.9X
Array with 30000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 19200 / 19206 25.6 39.1 1.0X
ExternalAppendOnlyUnsafeRowArray 19558 / 19562 25.1 39.8 1.0X
Array with 100000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 5949 / 6028 17.2 58.1 1.0X
ExternalAppendOnlyUnsafeRowArray 6078 / 6138 16.8 59.4 1.0X
```
[B] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `UnsafeExternalSorter` when there is spilling of data
```
Spilling with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter 9239 / 9470 28.4 35.2 1.0X
ExternalAppendOnlyUnsafeRowArray 8857 / 8909 29.6 33.8 1.0X
Spilling with 10000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter 4 / 5 39.3 25.5 1.0X
ExternalAppendOnlyUnsafeRowArray 5 / 6 29.8 33.5 0.8X
```
Author: Tejas Patil <tejasp@fb.com>
Closes#16909 from tejasapatil/SPARK-13450_smb_buffer_oom.
## What changes were proposed in this pull request?
We should restrict the nested level of a view, to avoid stack overflow exception during the view resolution.
## How was this patch tested?
Add new test case in `SQLViewSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17241 from jiangxb1987/view-depth.
## What changes were proposed in this pull request?
As timezone setting can also affect partition values, it works for all formats, we should make it clear.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#17281 from ueshin/issues/SPARK-19817.
## What changes were proposed in this pull request?
This PR fixes the following problem:
````
Seq((1, 2)).toDF("a1", "a2").createOrReplaceTempView("a")
Seq[(java.lang.Integer, java.lang.Integer)]((1, null)).toDF("b1", "b2").createOrReplaceTempView("b")
// The expected result is 1 row of (1,2) as shown in the next statement.
sql("select * from a where a1 not in (select b1 from b where b2 = a2)").show
+---+---+
| a1| a2|
+---+---+
+---+---+
sql("select * from a where a1 not in (select b1 from b where b2 = 2)").show
+---+---+
| a1| a2|
+---+---+
| 1| 2|
+---+---+
````
There are a number of scenarios to consider:
1. When the correlated predicate yields a match (i.e., B.B2 = A.A2)
1.1. When the NOT IN expression yields a match (i.e., A.A1 = B.B1)
1.2. When the NOT IN expression yields no match (i.e., A.A1 = B.B1 returns false)
1.3. When A.A1 is null
1.4. When B.B1 is null
1.4.1. When A.A1 is not null
1.4.2. When A.A1 is null
2. When the correlated predicate yields no match (i.e.,B.B2 = A.A2 is false or unknown)
2.1. When B.B2 is null and A.A2 is null
2.2. When B.B2 is null and A.A2 is not null
2.3. When the value of A.A2 does not match any of B.B2
````
A.A1 A.A2 B.B1 B.B2
----- ----- ----- -----
1 1 1 1 (1.1)
2 1 (1.2)
null 1 (1.3)
1 3 null 3 (1.4.1)
null 3 (1.4.2)
1 null 1 null (2.1)
null 2 (2.2 & 2.3)
````
We can divide the evaluation of the above correlated NOT IN subquery into 2 groups:-
Group 1: The rows in A when there is a match from the correlated predicate (A.A1 = B.B1)
In this case, the result of the subquery is not empty and the semantics of the NOT IN depends solely on the evaluation of the equality comparison of the columns of NOT IN, i.e., A1 = B1, which says
- If A.A1 is null, the row is filtered (1.3 and 1.4.2)
- If A.A1 = B.B1, the row is filtered (1.1)
- If B.B1 is null, any rows of A in the same group (A.A2 = B.B2) is filtered (1.4.1 & 1.4.2)
- Otherwise, the row is qualified.
Hence, in this group, the result is the row from (1.2).
Group 2: The rows in A when there is no match from the correlated predicate (A.A2 = B.B2)
In this case, all the rows in A, including the rows where A.A1, are qualified because the subquery returns an empty set and by the semantics of the NOT IN, all rows from the parent side qualifies as the result set, that is, the rows from (2.1, 2.2 and 2.3).
In conclusion, the correct result set of the above query is
````
A.A1 A.A2
----- -----
2 1 (1.2)
1 null (2.1)
null 2 (2.2 & 2.3)
````
## How was this patch tested?
unit tests, regression tests, and new test cases focusing on the problem being fixed.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#17294 from nsyca/18966.
## What changes were proposed in this pull request?
The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan.
This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained.
## How was this patch tested?
Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17278 from hvanhovell/SPARK-19933.
## What changes were proposed in this pull request?
We should support the statement `SHOW TABLE EXTENDED LIKE 'table_identifier' PARTITION(partition_spec)`, just like that HIVE does.
When partition is specified, the `SHOW TABLE EXTENDED` command should output the information of the partitions instead of the tables.
Note that in this statement, we require exact matched partition spec. For example:
```
CREATE TABLE show_t1(a String, b Int) PARTITIONED BY (c String, d String);
ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) PARTITION (c='Us', d=22);
-- Output the extended information of Partition(c='Us', d=1)
SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1);
-- Throw an AnalysisException
SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us');
```
## How was this patch tested?
Add new test sqls in file `show-tables.sql`.
Add new test case in `DDLSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16373 from jiangxb1987/show-partition-extended.
## What changes were proposed in this pull request?
We currently cannot use aliases in SQL function calls. This is inconvenient when you try to create a struct. This SQL query for example `select struct(1, 2) st`, will create a struct with column names `col1` and `col2`. This is even more problematic when we want to append a field to an existing struct. For example if we want to a field to struct `st` we would issue the following SQL query `select struct(st.*, 1) as st from src`, the result will be struct `st` with an a column with a non descriptive name `col3` (if `st` itself has 2 fields).
This PR proposes to change this by allowing the use of aliased expression in function parameters. For example `select struct(1 as a, 2 as b) st`, will create a struct with columns `a` & `b`.
## How was this patch tested?
Added a test to `ExpressionParserSuite` and added a test file for `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17245 from hvanhovell/SPARK-19850.
## What changes were proposed in this pull request?
This JIRA is a follow up work after [SPARK-19583](https://issues.apache.org/jira/browse/SPARK-19583)
As we discussed in that [PR](https://github.com/apache/spark/pull/16938)
The following DDL for datasource table with an non-existent location should work:
```
CREATE TABLE ... (PARTITIONED BY ...) LOCATION path
```
Currently it will throw exception that path not exists for datasource table for datasource table
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#17055 from windpiger/CTDataSourcePathNotExists.
## What changes were proposed in this pull request?
In spark SQL, map type can't be used in equality test/comparison, and `Intersect`/`Except`/`Distinct` do need equality test for all columns, we should not allow map type in `Intersect`/`Except`/`Distinct`.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17236 from cloud-fan/map.
## Summary of changes
Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties.
- Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf
- Add schemaPreservesCase field to CatalogTable (set to false when schema can't
successfully be read from Hive table props)
- Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is
false, depending on spark.sql.hive.caseSensitiveInferenceMode
- Add alterTableSchema() method to the ExternalCatalog interface
- Add HiveSchemaInferenceSuite tests
- Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as
HiveMetastoreCatalog.mergeWithMetastoreSchema
- Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite
[JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)
## How was this patch tested?
The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API.
Author: Budde <budde@amazon.com>
Closes#16944 from budde/SPARK-19611.
## What changes were proposed in this pull request?
`watermark` should not be negative. This behavior is invalid, check it before real run.
## How was this patch tested?
add new unit test.
Author: uncleGen <hustyugm@gmail.com>
Author: dylon <hustyugm@gmail.com>
Closes#17202 from uncleGen/SPARK-19861.
## What changes were proposed in this pull request?
Today, we compare the whole path when deciding if a file is new in the FileSource for structured streaming. However, this would cause false negatives in the case where the path has changed in a cosmetic way (i.e. changing `s3n` to `s3a`).
This patch adds an option `fileNameOnly` that causes the new file check to be based only on the filename (but still store the whole path in the log).
## Usage
```scala
spark
.readStream
.option("fileNameOnly", true)
.text("s3n://bucket/dir1/dir2")
.writeStream
...
```
## How was this patch tested?
Added a test case
Author: Liwei Lin <lwlin7@gmail.com>
Closes#17120 from lw-lin/filename-only.
## What changes were proposed in this pull request?
If we create a external datasource table with a non-qualified location , we should qualified it to store in catalog.
```
CREATE TABLE t(a string)
USING parquet
LOCATION '/path/xx'
CREATE TABLE t1(a string, b string)
USING parquet
PARTITIONED BY(b)
LOCATION '/path/xx'
```
when we get the table from catalog, the location should be qualified, e.g.'file:/path/xxx'
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#17095 from windpiger/tablepathQualified.
### What changes were proposed in this pull request?
So far, the test cases in DDLSuites only verify the behaviors of InMemoryCatalog. That means, they do not cover the scenarios using HiveExternalCatalog. Thus, we need to improve the existing test suite to run these cases using Hive metastore.
When porting these test cases, a bug of `SET LOCATION` is found. `path` is not set when the location is changed.
After this PR, a few changes are made, as summarized below,
- `DDLSuite` becomes an abstract class. Both `InMemoryCatalogedDDLSuite` and `HiveCatalogedDDLSuite` extend it. `InMemoryCatalogedDDLSuite` is using `InMemoryCatalog`. `HiveCatalogedDDLSuite` is using `HiveExternalCatalog`.
- `InMemoryCatalogedDDLSuite` contains all the existing test cases in `DDLSuite`.
- `HiveCatalogedDDLSuite` contains a subset of `DDLSuite`. The following test cases are excluded:
1. The following test cases only make sense for `InMemoryCatalog`:
```
test("desc table for parquet data source table using in-memory catalog")
test("create a managed Hive source table") {
test("create an external Hive source table")
test("Create Hive Table As Select")
```
2. The following test cases are unable to be ported because we are unable to alter table provider when using Hive metastore. In the future PRs we need to improve the test cases so that altering table provider is not needed:
```
test("alter table: set location (datasource table)")
test("alter table: set properties (datasource table)")
test("alter table: unset properties (datasource table)")
test("alter table: set serde (datasource table)")
test("alter table: set serde partition (datasource table)")
test("alter table: change column (datasource table)")
test("alter table: add partition (datasource table)")
test("alter table: drop partition (datasource table)")
test("alter table: rename partition (datasource table)")
test("drop table - data source table")
```
**TODO** : in the future PRs, we need to remove `HiveDDLSuite` and move the test cases to either `DDLSuite`, `InMemoryCatalogedDDLSuite` or `HiveCatalogedDDLSuite`.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16592 from gatorsmile/refactorDDLSuite.
## What changes were proposed in this pull request?
**The Problem**
There is a file stream source option called maxFileAge which limits how old the files can be, relative the latest file that has been seen. This is used to limit the files that need to be remembered as "processed". Files older than the latest processed files are ignored. This values is by default 7 days.
This causes a problem when both
latestFirst = true
maxFilesPerTrigger > total files to be processed.
Here is what happens in all combinations
1) latestFirst = false - Since files are processed in order, there wont be any unprocessed file older than the latest processed file. All files will be processed.
2) latestFirst = true AND maxFilesPerTrigger is not set - The maxFileAge thresholding mechanism takes one batch initialize. If maxFilesPerTrigger is not, then all old files get processed in the first batch, and so no file is left behind.
3) latestFirst = true AND maxFilesPerTrigger is set to X - The first batch process the latest X files. That sets the threshold latest file - maxFileAge, so files older than this threshold will never be considered for processing.
The bug is with case 3.
**The Solution**
Ignore `maxFileAge` when both `maxFilesPerTrigger` and `latestFirst` are set.
## How was this patch tested?
Regression test in `FileStreamSourceSuite`
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#17153 from brkyvz/maxFileAge.
## What changes were proposed in this pull request?
This PR proposes to add an API that loads `DataFrame` from `Dataset[String]` storing csv.
It allows pre-processing before loading into CSV, which means allowing a lot of workarounds for many narrow cases, for example, as below:
- Case 1 - pre-processing
```scala
val df = spark.read.text("...")
// Pre-processing with this.
spark.read.csv(df.as[String])
```
- Case 2 - use other input formats
```scala
val rdd = spark.sparkContext.newAPIHadoopFile("/file.csv.lzo",
classOf[com.hadoop.mapreduce.LzoTextInputFormat],
classOf[org.apache.hadoop.io.LongWritable],
classOf[org.apache.hadoop.io.Text])
val stringRdd = rdd.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength))
spark.read.csv(stringRdd.toDS)
```
## How was this patch tested?
Added tests in `CSVSuite` and build with Scala 2.10.
```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16854 from HyukjinKwon/SPARK-15463.
Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a method cloneSession() which creates a new SparkSession with a copy of the parent's SessionState.
Subsequent changes to base session are not propagated to cloned session, clone is independent after creation.
If the base is changed after clone has been created, say user registers new UDF, then the new UDF will not be available inside the clone. Same goes for configs and temp tables.
Unit tests
Author: Kunal Khamar <kkhamar@outlook.com>
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16826 from kunalkhamar/fork-sparksession.
## What changes were proposed in this pull request?
Add a output mode parameter to `flatMapGroupsWithState` and just define `mapGroupsWithState` as `flatMapGroupsWithState(Update)`.
`UnsupportedOperationChecker` is modified to disallow unsupported cases.
- Batch mapGroupsWithState or flatMapGroupsWithState is always allowed.
- For streaming (map/flatMap)GroupsWithState, see the following table:
| Operators | Supported Query Output Mode |
| ------------- | ------------- |
| flatMapGroupsWithState(Update) without aggregation | Update |
| flatMapGroupsWithState(Update) with aggregation | None |
| flatMapGroupsWithState(Append) without aggregation | Append |
| flatMapGroupsWithState(Append) before aggregation | Append, Update, Complete |
| flatMapGroupsWithState(Append) after aggregation | None |
| Multiple flatMapGroupsWithState(Append)s | Append |
| Multiple mapGroupsWithStates | None |
| Mxing mapGroupsWithStates and flatMapGroupsWithStates | None |
| Other cases of multiple flatMapGroupsWithState | None |
## How was this patch tested?
The added unit tests. Here are the tests related to (map/flatMap)GroupsWithState:
```
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation: supported (1 millisecond)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Append)s on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Update)s on batch relation: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in update mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in append mode: not supported (7 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Append mode: not supported (11 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Update mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in update mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Update mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Complete mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Update mode: not supported (4 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation in complete mode: not supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Append output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Update output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Append output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Update output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on streaming relation and all are in append mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on s streaming relation but some are not in append mode: not supported (7 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in append mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in complete mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Update mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Complete mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - multiple mapGroupsWithStates on streaming relation and all are in append mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - mixing mapGroupsWithStates and flatMapGroupsWithStates on streaming relation: not supported (4 milliseconds)
```
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17197 from zsxwing/mapgroups-check.
## What changes were proposed in this pull request?
Fix for SQL round function that modifies original column when underlying data frame is created from a local product.
import org.apache.spark.sql.functions._
case class NumericRow(value: BigDecimal)
val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"))))
df.show()
+--------------------+
| value|
+--------------------+
|1.234567890000000000|
+--------------------+
df.withColumn("value_rounded", round('value)).show()
// before
+--------------------+-------------+
| value|value_rounded|
+--------------------+-------------+
|1.000000000000000000| 1|
+--------------------+-------------+
// after
+--------------------+-------------+
| value|value_rounded|
+--------------------+-------------+
|1.234567890000000000| 1|
+--------------------+-------------+
## How was this patch tested?
New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite`
Author: Wojtek Szymanski <wk.szymanski@gmail.com>
Closes#17075 from wojtek-szymanski/SPARK-19727.
## What changes were proposed in this pull request?
Currently there are lots of places to make the path qualified, it is better to provide a function to do this, then the code will be more simple.
## How was this patch tested?
N/A
Author: windpiger <songjun@outlook.com>
Closes#17204 from windpiger/addQualifiledPathUtil.
### What changes were proposed in this pull request?
Observed by felixcheung in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later.
Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result.
```Scala
val df = spark.range(0, 10000, 1, 5)
val df2 = df.repartition(10)
assert(df2.coalesce(13).rdd.getNumPartitions == 5)
assert(df2.coalesce(7).rdd.getNumPartitions == 5)
assert(df2.coalesce(3).rdd.getNumPartitions == 3)
```
This PR is to fix the issue. We preserve shuffle-enabled Repartition.
### How was this patch tested?
Added a test case
Author: Xiao Li <gatorsmile@gmail.com>
Closes#16933 from gatorsmile/CollapseRepartition.
## What changes were proposed in this pull request?
Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.
## How was this patch tested?
Update the related test cases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17210 from jiangxb1987/SubqueryAlias.
## What changes were proposed in this pull request?
Reorder the joins using a dynamic programming algorithm (Selinger paper):
First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.
When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
```
level 1: p({A}), p({B}), p({C}), p({D})
level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
level 4: p({A, B, C, D})
```
where p({A, B, C, D}) is the final output plan.
For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.
## How was this patch tested?
add test cases
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#17138 from wzhfy/joinReorder.
## What changes were proposed in this pull request?
Make the `SET mapreduce.job.reduces` automatically converted to `spark.sql.shuffle.partitions`, it's similar to `SET mapred.reduce.tasks`.
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#17020 from wangyum/SPARK-19693.
## What changes were proposed in this pull request?
The new watermark should override the old one. Otherwise, we just pick up the first column which has a watermark, it may be unexpected.
## How was this patch tested?
The new test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17199 from zsxwing/SPARK-19859.
## What changes were proposed in this pull request?
`StreamingDeduplicateExec.watermarkPredicate` should filter based on keys. Otherwise, it may generate a wrong answer if the watermark column in `keyExpression` has a different position in the row.
`StateStoreSaveExec` has the same codes but its parent can makes sure the watermark column positions in `keyExpression` and `row` are the same.
## How was this patch tested?
The added test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17183 from zsxwing/SPARK-19841.
## What changes were proposed in this pull request?
Disallow cyclic view references, a cyclic view reference may be created by the following queries:
```
CREATE VIEW testView AS SELECT id FROM tbl
CREATE VIEW testView2 AS SELECT id FROM testView
ALTER VIEW testView AS SELECT * FROM testView2
```
In the above example, a reference cycle (testView -> testView2 -> testView) exsits.
We disallow cyclic view references by checking that in ALTER VIEW command, when the `analyzedPlan` contains the same `View` node with the altered view, we should prevent the behavior and throw an AnalysisException.
## How was this patch tested?
Test by `SQLViewSuite.test("correctly handle a cyclic view reference")`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17152 from jiangxb1987/cyclic-view.
## What changes were proposed in this pull request?
When un-cache a table, we should not only remove the cache entry for this table, but also un-cache any other cached plans that refer to this table.
This PR also includes some refactors:
1. use `java.util.LinkedList` to store the cache entries, so that it's safer to remove elements while iterating
2. rename `invalidateCache` to `recacheByPlan`, which is more obvious about what it does.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17097 from cloud-fan/cache.
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `to_json` in SQL.
## How was this patch tested?
Added tests in `JsonFunctionsSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#16981 from maropu/SPARK-19637.
## What changes were proposed in this pull request?
Currently in DynamicPartitionWriteTask, when we get the paritionPath of a parition, we just escape the partition value, not escape the partition name.
this will cause some problems for some special partition name situation, for example :
1) if the partition name contains '%' etc, there will be two partition path created in the filesytem, one is for escaped path like '/path/a%25b=1', another is for unescaped path like '/path/a%b=1'.
and the data inserted stored in unescaped path, while the show partitions table will return 'a%25b=1' which the partition name is escaped. So here it is not consist. And I think the data should be stored in the escaped path in filesystem, which Hive2.0.0 also have the same action.
2) if the partition name contains ':', there will throw exception that new Path("/path","a:b"), this is illegal which has a colon in the relative path.
```
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: a:b
at org.apache.hadoop.fs.Path.initialize(Path.java:205)
at org.apache.hadoop.fs.Path.<init>(Path.java:171)
at org.apache.hadoop.fs.Path.<init>(Path.java:88)
... 48 elided
Caused by: java.net.URISyntaxException: Relative path in absolute URI: a:b
at java.net.URI.checkPath(URI.java:1823)
at java.net.URI.<init>(URI.java:745)
at org.apache.hadoop.fs.Path.initialize(Path.java:202)
... 50 more
```
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#17173 from windpiger/fixDatasourceSpecialCharPartitionName.
## What changes were proposed in this pull request?
Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect.
We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not.
We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample.
## How was this patch tested?
Added test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16696 from wzhfy/limitEstimation.
## What changes were proposed in this pull request?
Bugfix for reading empty file with CSV data source. Instead of throwing `NoSuchElementException`, an empty data frame is returned.
## How was this patch tested?
Added new unit test in `org.apache.spark.sql.execution.datasources.csv.CSVSuite`
Author: Wojtek Szymanski <wk.szymanski@gmail.com>
Closes#17068 from wojtek-szymanski/SPARK-19709.
## What changes were proposed in this pull request?
Currently we don't explicitly forbid the following behaviors:
1. The statement CREATE VIEW AS INSERT INTO throws the following exception:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: at least one column must be specified for the table;
scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: The number of columns produced by the SELECT clause (num: `0`) does not match the number of column names specified by CREATE VIEW (num: `2`).;
```
2. The statement INSERT INTO view VALUES throws the following exception from checkAnalysis:
```
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
'InsertIntoTable View (`default`.`testView`, [a#16,b#17]), false, false
+- LocalRelation [col1#14, col2#15]
```
After this PR, the behavior changes to:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;
scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: `default`.`testView` is a view, inserting into a view is not allowed;
```
## How was this patch tested?
Add a new test case in `SparkSqlParserSuite`;
Update the corresponding test case in `SQLViewSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17125 from jiangxb1987/insert-with-view.
## What changes were proposed in this pull request?
Currently we treat the location of table/partition/database as URI string.
It will be safer if we can make the type of location as java.net.URI.
In this PR, there are following classes changes:
**1. CatalogDatabase**
```
case class CatalogDatabase(
name: String,
description: String,
locationUri: String,
properties: Map[String, String])
--->
case class CatalogDatabase(
name: String,
description: String,
locationUri: URI,
properties: Map[String, String])
```
**2. CatalogStorageFormat**
```
case class CatalogStorageFormat(
locationUri: Option[String],
inputFormat: Option[String],
outputFormat: Option[String],
serde: Option[String],
compressed: Boolean,
properties: Map[String, String])
---->
case class CatalogStorageFormat(
locationUri: Option[URI],
inputFormat: Option[String],
outputFormat: Option[String],
serde: Option[String],
compressed: Boolean,
properties: Map[String, String])
```
Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally.
Here list some operation related location:
**1. whitespace in the location**
e.g. `/a/b c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b c/d`,
and the real path in the FileSystem also show `/a/b c/d`
**2. colon(:) in the location**
e.g. `/a/b:c/d`
For both table location and partition location,
when `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b:c/d'` ,
**In linux file system**
`DESC EXTENDED t ` show the location is `/a/b:c/d`,
and the real path in the FileSystem also show `/a/b:c/d`
**in HDFS** throw exception:
`java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.`
**while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1`
then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`,
and the real path in the FileSystem also show `/xxx/a=a%3Ab`
**3. percent sign(%) in the location**
e.g. `/a/b%c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b%c/d`,
and the real path in the FileSystem also show `/a/b%c/d`
**4. encoded(%25) in the location**
e.g. `/a/b%25c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%25c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b%25c/d`,
and the real path in the FileSystem also show `/a/b%25c/d`
**while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1`
then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`,
and the real path in the FileSystem also show `/xxx/a=%2525`
**Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173)
### Summary:
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION path`,
the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ).
`DataBase` also have the same logic with `CREATE TABLE`
while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem`
In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString`
which transfrom `str to uri `or `uri to str`.
for example:
```
val str = '/a/b c/d'
val uri = new Path(str).toUri --> '/a/b%20c/d'
val strFromUri = new Path(uri).toString -> '/a/b c/d'
```
when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri `
## How was this patch tested?
unit test added.
The `current master branch` also `passed all the test cases` added in this PR by a litter change.
https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764
here `toURI` -> `toString` when test in master branch.
This can show that this PR is transparent for user.
Author: windpiger <songjun@outlook.com>
Closes#17149 from windpiger/changeStringToURI.
## What changes were proposed in this pull request?
This PR proposes to both,
**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**
Currently, it only reads the single row when the input is a json array. So, the codes below:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| [1]|
+--------------------+
```
This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| null|
+--------------------+
```
**Support json arrays in `from_json` with `ArrayType` as the schema.**
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```
prints
```
+-------------------+
|jsontostruct(array)|
+-------------------+
| [[1], [2]]|
+-------------------+
```
## How was this patch tested?
Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16929 from HyukjinKwon/disallow-array.
## What changes were proposed in this pull request?
This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure.
## How was this patch tested?
Added tests in `LiteralExpressionSuite`
Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#16610 from maropu/SPARK-19254.
## What changes were proposed in this pull request?
"DataFrameCallbackSuite.execute callback functions when a DataFrame action failed" sets the log level to "fatal" but doesn't recover it. Hence, tests running after it won't output any logs except fatal logs.
This PR uses `testQuietly` instead to avoid changing the log level.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17156 from zsxwing/SPARK-19816.
## What changes were proposed in this pull request?
[SPARK-19617](https://issues.apache.org/jira/browse/SPARK-19617) changed `HDFSMetadataLog` to enable interrupts when using the local file system. However, now we hit [HADOOP-12074](https://issues.apache.org/jira/browse/HADOOP-12074): `Shell.runCommand` converts `InterruptedException` to `new IOException(ie.toString())` before Hadoop 2.8. This is the Hadoop patch to fix HADOOP-1207: 95c73d49b1
This PR adds new logic to handle the following cases related to `InterruptedException`.
- Check if the message of IOException starts with `java.lang.InterruptedException`. If so, treat it as `InterruptedException`. This is for pre-Hadoop 2.8.
- Treat `InterruptedIOException` as `InterruptedException`. This is for Hadoop 2.8+ and other places that may throw `InterruptedIOException` when the thread is interrupted.
## How was this patch tested?
The new unit test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17044 from zsxwing/SPARK-19718.
## What changes were proposed in this pull request?
This is a follow-up pr of #16308 and #16750.
This pr enables timezone support in partition values.
We should use `timeZone` option introduced at #16750 to parse/format partition values of the `TimestampType`.
For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT` which will be used for partition values, the values written by the default timezone option, which is `"GMT"` because the session local timezone is `"GMT"` here, are:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")
scala> val df = Seq((1, new java.sql.Timestamp(1451606400000L))).toDF("i", "ts")
df: org.apache.spark.sql.DataFrame = [i: int, ts: timestamp]
scala> df.show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
scala> df.write.partitionBy("ts").save("/path/to/gmtpartition")
```
```sh
$ ls /path/to/gmtpartition/
_SUCCESS ts=2016-01-01 00%3A00%3A00
```
whereas setting the option to `"PST"`, they are:
```scala
scala> df.write.option("timeZone", "PST").partitionBy("ts").save("/path/to/pstpartition")
```
```sh
$ ls /path/to/pstpartition/
_SUCCESS ts=2015-12-31 16%3A00%3A00
```
We can properly read the partition values if the session local timezone and the timezone of the partition values are the same:
```scala
scala> spark.read.load("/path/to/gmtpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
```
And even if the timezones are different, we can properly read the values with setting corrent timezone option:
```scala
// wrong result
scala> spark.read.load("/path/to/pstpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2015-12-31 16:00:00|
+---+-------------------+
// correct result
scala> spark.read.option("timeZone", "PST").load("/path/to/pstpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
```
## How was this patch tested?
Existing tests and added some tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#17053 from ueshin/issues/SPARK-18939.