## What changes were proposed in this pull request?
`QueryExecution.toString` currently captures `java.lang.Throwable`s; this is far from a best practice and can lead to confusing situation or invalid application states. This PR fixes this by only capturing `AnalysisException`s.
## How was this patch tested?
Added a `QueryExecutionSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15760 from hvanhovell/SPARK-18259.
## What changes were proposed in this pull request?
This patch improves error reporting for FileStressSuite, when there is an error in Spark itself (not user code). This works by simply tightening the exception verification, and gets rid of the unnecessary thread for starting the stream.
Also renamed the class FileStreamStressSuite to make it more obvious it is a streaming suite.
## How was this patch tested?
This is a test only change and I manually verified error reporting by injecting some bug in the addBatch code for FileStreamSink.
Author: Reynold Xin <rxin@databricks.com>
Closes#15757 from rxin/SPARK-18257.
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.
## How was this patch tested?
Should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#15750 from rxin/SPARK-18244.
## What changes were proposed in this pull request?
This PR adds a new hash-based aggregate operator named `ObjectHashAggregateExec` that supports `TypedImperativeAggregate`, which may use arbitrary Java objects as aggregation states. Please refer to the [design doc](https://issues.apache.org/jira/secure/attachment/12834260/%5BDesign%20Doc%5D%20Support%20for%20Arbitrary%20Aggregation%20States.pdf) attached in [SPARK-17949](https://issues.apache.org/jira/browse/SPARK-17949) for more details about it.
The major benefit of this operator is better performance when evaluating `TypedImperativeAggregate` functions, especially when there are relatively few distinct groups. Functions like Hive UDAFs, `collect_list`, and `collect_set` may also benefit from this after being migrated to `TypedImperativeAggregate`.
The following feature flag is introduced to enable or disable the new aggregate operator:
- Name: `spark.sql.execution.useObjectHashAggregateExec`
- Default value: `true`
We can also configure the fallback threshold using the following SQL operation:
- Name: `spark.sql.objectHashAggregate.sortBased.fallbackThreshold`
- Default value: 128
Fallback to sort-based aggregation when more than 128 distinct groups are accumulated in the aggregation hash map. This number is intentionally made small to avoid GC problems since aggregation buffers of this operator may contain arbitrary Java objects.
This may be improved by implementing size tracking for this operator, but that can be done in a separate PR.
Code generation and size tracking are planned to be implemented in follow-up PRs.
## Benchmark results
### `ObjectHashAggregateExec` vs `SortAggregateExec`
The first benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating `typed_count`, a testing `TypedImperativeAggregate` version of the SQL `count` function.
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by 31251 / 31908 3.4 298.0 1.0X
object agg w/ group by w/o fallback 6903 / 7141 15.2 65.8 4.5X
object agg w/ group by w/ fallback 20945 / 21613 5.0 199.7 1.5X
sort agg w/o group by 4734 / 5463 22.1 45.2 6.6X
object agg w/o group by w/o fallback 4310 / 4529 24.3 41.1 7.3X
```
The next benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating the Spark native version of `percentile_approx`.
Note that `percentile_approx` is so heavy an aggregate function that the bottleneck of the benchmark is evaluating the aggregate function itself rather than the aggregate operator since I couldn't run a large scale benchmark on my laptop. That's why the results are so close and looks counter-intuitive (aggregation with grouping is even faster than that aggregation without grouping).
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by 3418 / 3530 0.6 1630.0 1.0X
object agg w/ group by w/o fallback 3210 / 3314 0.7 1530.7 1.1X
object agg w/ group by w/ fallback 3419 / 3511 0.6 1630.1 1.0X
sort agg w/o group by 4336 / 4499 0.5 2067.3 0.8X
object agg w/o group by w/o fallback 4271 / 4372 0.5 2036.7 0.8X
```
### Hive UDAF vs Spark AF
This benchmark compares the following two kinds of aggregate functions:
- "hive udaf": Hive implementation of `percentile_approx`, without partial aggregation supports, evaluated using `SortAggregateExec`.
- "spark af": Spark native implementation of `percentile_approx`, with partial aggregation support, evaluated using `ObjectHashAggregateExec`
The performance differences are mostly due to faster implementation and partial aggregation support in the Spark native version of `percentile_approx`.
This benchmark basically shows the performance differences between the worst case, where an aggregate function without partial aggregation support is evaluated using `SortAggregateExec`, and the best case, where a `TypedImperativeAggregate` with partial aggregation support is evaluated using `ObjectHashAggregateExec`.
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
hive udaf w/o group by 5326 / 5408 0.0 81264.2 1.0X
spark af w/o group by 93 / 111 0.7 1415.6 57.4X
hive udaf w/ group by 3804 / 3946 0.0 58050.1 1.4X
spark af w/ group by w/o fallback 71 / 90 0.9 1085.7 74.8X
spark af w/ group by w/ fallback 98 / 111 0.7 1501.6 54.1X
```
### Real world benchmark
We also did a relatively large benchmark using a real world query involving `percentile_approx`:
- Hive UDAF implementation, sort-based aggregation, w/o partial aggregation support
24.77 minutes
- Native implementation, sort-based aggregation, w/ partial aggregation support
4.64 minutes
- Native implementation, object hash aggregator, w/ partial aggregation support
1.80 minutes
## How was this patch tested?
New unit tests and randomized test cases are added in `ObjectAggregateFunctionSuite`.
Author: Cheng Lian <lian@databricks.com>
Closes#15590 from liancheng/obj-hash-agg.
### What changes were proposed in this pull request?
When `FilterExec` contains `isNotNull`, which could be inferred and pushed down or users specified, we convert the nullability of the involved columns if the top-layer expression is null-intolerant. However, this is not correct, if the top-layer expression is not a leaf expression, it could still tolerate the null when it has null-tolerant child expressions.
For example, `cast(coalesce(a#5, a#15) as double)`. Although `cast` is a null-intolerant expression, but obviously`coalesce` is null-tolerant. Thus, it could eat null.
When the nullability is wrong, we could generate incorrect results in different cases. For example,
``` Scala
val df1 = Seq((1, 2), (2, 3)).toDF("a", "b")
val df2 = Seq((2, 5), (3, 4)).toDF("a", "c")
val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0)
val df3 = Seq((3, 1)).toDF("a", "d")
joinedDf.join(df3, "a").show
```
The optimized plan is like
```
Project [a#29, b#30, c#31, d#42]
+- Join Inner, (a#29 = a#41)
:- Project [cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int) AS a#29, cast(coalesce(cast(b#6 as double), 0.0) as int) AS b#30, cast(coalesce(cast(c#16 as double), 0.0) as int) AS c#31]
: +- Filter isnotnull(cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int))
: +- Join FullOuter, (a#5 = a#15)
: :- LocalRelation [a#5, b#6]
: +- LocalRelation [a#15, c#16]
+- LocalRelation [a#41, d#42]
```
Without the fix, it returns an empty result. With the fix, it can return a correct answer:
```
+---+---+---+---+
| a| b| c| d|
+---+---+---+---+
| 3| 0| 4| 1|
+---+---+---+---+
```
### How was this patch tested?
Added test cases to verify the nullability changes in FilterExec. Also added a test case for verifying the reported incorrect result.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15523 from gatorsmile/nullabilityFilterExec.
## What changes were proposed in this pull request?
In Spark 1.6 and earlier, we can drop the database we are using. In Spark 2.0, native implementation prevent us from dropping current database, which may break some old queries. This PR would re-enable the feature.
## How was this patch tested?
one new unit test in `SessionCatalogSuite`.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#15011 from adrian-wang/dropcurrent.
## What changes were proposed in this pull request?
This PR proposes to change the documentation for functions. Please refer the discussion from https://github.com/apache/spark/pull/15513
The changes include
- Re-indent the documentation
- Add examples/arguments in `extended` where the arguments are multiple or specific format (e.g. xml/ json).
For examples, the documentation was updated as below:
### Functions with single line usage
**Before**
- `pow`
``` sql
Usage: pow(x1, x2) - Raise x1 to the power of x2.
Extended Usage:
> SELECT pow(2, 3);
8.0
```
- `current_timestamp`
``` sql
Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
Extended Usage:
No example for current_timestamp.
```
**After**
- `pow`
``` sql
Usage: pow(expr1, expr2) - Raises `expr1` to the power of `expr2`.
Extended Usage:
Examples:
> SELECT pow(2, 3);
8.0
```
- `current_timestamp`
``` sql
Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
Extended Usage:
No example/argument for current_timestamp.
```
### Functions with (already) multiple line usage
**Before**
- `approx_count_distinct`
``` sql
Usage: approx_count_distinct(expr) - Returns the estimated cardinality by HyperLogLog++.
approx_count_distinct(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++
with relativeSD, the maximum estimation error allowed.
Extended Usage:
No example for approx_count_distinct.
```
- `percentile_approx`
``` sql
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate
percentile array of column `col` at the given percentage array. Each value of the
percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is
a positive integer literal which controls approximation accuracy at the cost of memory.
Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of
the approximation.
Extended Usage:
No example for percentile_approx.
```
**After**
- `approx_count_distinct`
``` sql
Usage:
approx_count_distinct(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++.
`relativeSD` defines the maximum estimation error allowed.
Extended Usage:
No example/argument for approx_count_distinct.
```
- `percentile_approx`
``` sql
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
In this case, returns the approximate percentile array of column `col` at the given
percentage array.
Extended Usage:
Examples:
> SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
[10.0,10.0,10.0]
> SELECT percentile_approx(10.0, 0.5, 100);
10.0
```
## How was this patch tested?
Manually tested
**When examples are multiple**
``` sql
spark-sql> describe function extended reflect;
Function: reflect
Class: org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection
Usage: reflect(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.
Extended Usage:
Examples:
> SELECT reflect('java.util.UUID', 'randomUUID');
c33fb387-8500-4bfa-81d2-6e0e3e930df2
> SELECT reflect('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2');
a5cf6c42-0c85-418f-af6c-3e4e5b1328f2
```
**When `Usage` is in single line**
``` sql
spark-sql> describe function extended min;
Function: min
Class: org.apache.spark.sql.catalyst.expressions.aggregate.Min
Usage: min(expr) - Returns the minimum value of `expr`.
Extended Usage:
No example/argument for min.
```
**When `Usage` is already in multiple lines**
``` sql
spark-sql> describe function extended percentile_approx;
Function: percentile_approx
Class: org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
In this case, returns the approximate percentile array of column `col` at the given
percentage array.
Extended Usage:
Examples:
> SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
[10.0,10.0,10.0]
> SELECT percentile_approx(10.0, 0.5, 100);
10.0
```
**When example/argument is missing**
``` sql
spark-sql> describe function extended rank;
Function: rank
Class: org.apache.spark.sql.catalyst.expressions.Rank
Usage:
rank() - Computes the rank of a value in a group of values. The result is one plus the number
of rows preceding or equal to the current row in the ordering of the partition. The values
will produce gaps in the sequence.
Extended Usage:
No example/argument for rank.
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15677 from HyukjinKwon/SPARK-17963-1.
## What changes were proposed in this pull request?
Due to a limitation of hive metastore(table location must be directory path, not file path), we always store `path` for data source table in storage properties, instead of the `locationUri` field. However, we should not expose this difference to `CatalogTable` level, but just treat it as a hack in `HiveExternalCatalog`, like we store table schema of data source table in table properties.
This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, both data source table and hive serde table should use the `locationUri` field.
This PR also unifies the way we handle default table location for managed table. Previously, the default table location of hive serde managed table is set by external catalog, but the one of data source table is set by command. After this PR, we follow the hive way and the default table location is always set by external catalog.
For managed non-file-based tables, we will assign a default table location and create an empty directory for it, the table location will be removed when the table is dropped. This is reasonable as metastore doesn't care about whether a table is file-based or not, and an empty table directory has no harm.
For external non-file-based tables, ideally we can omit the table location, but due to a hive metastore issue, we will assign a random location to it, and remove it right after the table is created. See SPARK-15269 for more details. This is fine as it's well isolated in `HiveExternalCatalog`.
To keep the existing behaviour of the `path` option, in this PR we always add the `locationUri` to storage properties using key `path`, before passing storage properties to `DataSource` as data source options.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15024 from cloud-fan/path.
## What changes were proposed in this pull request?
RuntimeReplaceable is used to create aliases for expressions, but the way it deals with type coercion is pretty weird (each expression is responsible for how to handle type coercion, which does not obey the normal implicit type cast rules).
This patch simplifies its handling by allowing the analyzer to traverse into the actual expression of a RuntimeReplaceable.
## How was this patch tested?
- Correctness should be guaranteed by existing unit tests already
- Removed SQLCompatibilityFunctionSuite and moved it sql-compatibility-functions.sql
- Added a new test case in sql-compatibility-functions.sql for verifying explain behavior.
Author: Reynold Xin <rxin@databricks.com>
Closes#15723 from rxin/SPARK-18214.
## What changes were proposed in this pull request?
When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.
However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.
See the unit tests below or JIRA for examples.
This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?
Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)
cc: rxin davies
Author: Xiangrui Meng <meng@databricks.com>
Closes#15567 from mengxr/SPARK-14393.
## What changes were proposed in this pull request?
Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.
This PR includes:
1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.
## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.
Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.
Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>
Closes#15718 from hvanhovell/SPARK-16839-2.
## What changes were proposed in this pull request?
Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#15610 from srowen/SPARK-18076.
## What changes were proposed in this pull request?
The PR fixes the bug that the QueryStartedEvent is not logged
the postToAll() in the original code is actually calling StreamingQueryListenerBus.postToAll() which has no listener at all....we shall post by sparkListenerBus.postToAll(s) and this.postToAll() to trigger local listeners as well as the listeners registered in LiveListenerBus
zsxwing
## How was this patch tested?
The following snapshot shows that QueryStartedEvent has been logged correctly
![image](https://cloud.githubusercontent.com/assets/678008/19821553/007a7d28-9d2d-11e6-9f13-49851559cdaa.png)
Author: CodingCat <zhunansjtu@gmail.com>
Closes#15675 from CodingCat/SPARK-18144.
## What changes were proposed in this pull request?
This patch adds support for all file formats in structured streaming sinks. This is actually a very small change thanks to all the previous refactoring done using the new internal commit protocol API.
## How was this patch tested?
Updated FileStreamSinkSuite to add test cases for json, text, and parquet.
Author: Reynold Xin <rxin@databricks.com>
Closes#15711 from rxin/SPARK-18192.
## What changes were proposed in this pull request?
When the metadata logs for various parts of Structured Streaming are stored on non-HDFS filesystems such as NFS or ext4, the HDFSMetadataLog class leaves hidden HDFS-style checksum (CRC) files in the log directory, one file per batch. This PR modifies HDFSMetadataLog so that it detects the use of a filesystem that doesn't use CRC files and removes the CRC files.
## How was this patch tested?
Modified an existing test case in HDFSMetadataLogSuite to check whether HDFSMetadataLog correctly removes CRC files on the local POSIX filesystem. Ran the entire regression suite.
Author: frreiss <frreiss@us.ibm.com>
Closes#15027 from frreiss/fred-17475.
## What changes were proposed in this pull request?
This patch adds a new commit protocol implementation ManifestFileCommitProtocol that follows the existing streaming flow, and uses it in FileStreamSink to consolidate the write path in structured streaming with the batch mode write path.
This deletes a lot of code, and would make it trivial to support other functionalities that are currently available in batch but not in streaming, including all file formats and bucketing.
## How was this patch tested?
Should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#15710 from rxin/SPARK-18025.
## What changes were proposed in this pull request?
This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.
It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.
The usage is as below:
``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```
``` bash
+--------+
| json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?
Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15354 from HyukjinKwon/SPARK-17764.
## What changes were proposed in this pull request?
Aggregation Without Window/GroupBy expressions will fail in `checkAnalysis`, the error message is a bit misleading, we should generate a more specific error message for this case.
For example,
```
spark.read.load("/some-data")
.withColumn("date_dt", to_date($"date"))
.withColumn("year", year($"date_dt"))
.withColumn("week", weekofyear($"date_dt"))
.withColumn("user_count", count($"userId"))
.withColumn("daily_max_in_week", max($"user_count").over(weeklyWindow))
)
```
creates the following output:
```
org.apache.spark.sql.AnalysisException: expression '`randomColumn`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
```
In the error message above, `randomColumn` doesn't appear in the query(acturally it's added by function `withColumn`), so the message is not enough for the user to address the problem.
## How was this patch tested?
Manually test
Before:
```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: expression 'tbl.`col`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
```
After:
```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: grouping expressions sequence is empty, and 'tbl.`col`' is not an aggregate function. Wrap '(count(col#231L) AS count(col)#239L)' in windowing function(s) or wrap 'tbl.`col`' in first() (or first_value) if you don't care which value you get.;;
```
Also add new test sqls in `group-by.sql`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15672 from jiangxb1987/groupBy-empty.
## What changes were proposed in this pull request?
Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.
This PR includes:
1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.
## How was this patch tested?
running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.
modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.
Credit goes to hvanhovell for assisting with this PR.
Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>
Closes#14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
## What changes were proposed in this pull request?
Currently an unqualified `getFunction(..)`call returns a wrong result; the returned function is shown as temporary function without a database. For example:
```
scala> sql("create function fn1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'")
res0: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.getFunction("fn1")
res1: org.apache.spark.sql.catalog.Function = Function[name='fn1', className='org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs', isTemporary='true']
```
This PR fixes this by adding database information to ExpressionInfo (which is used to store the function information).
## How was this patch tested?
Added more thorough tests to `CatalogSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15542 from hvanhovell/SPARK-17996.
## What changes were proposed in this pull request?
When multiple records have the minimum value, the answer of ApproximatePercentile is wrong.
## How was this patch tested?
add a test case
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#15641 from wzhfy/percentile.
## What changes were proposed in this pull request?
The test `when schema inference is turned on, should read partition data` should not delete files because the source maybe is listing files. This PR just removes the delete actions since they are not necessary.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15699 from zsxwing/SPARK-18030.
## What changes were proposed in this pull request?
### Problem
Iterative ML code may easily create query plans that grow exponentially. We found that query planning time also increases exponentially even when all the sub-plan trees are cached.
The following snippet illustrates the problem:
``` scala
(0 until 6).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) =>
println(s"== Iteration $iteration ==")
val time0 = System.currentTimeMillis()
val joined = plan.join(plan, "value").join(plan, "value").join(plan, "value").join(plan, "value")
joined.cache()
println(s"Query planning takes ${System.currentTimeMillis() - time0} ms")
joined.as[Int]
}
// == Iteration 0 ==
// Query planning takes 9 ms
// == Iteration 1 ==
// Query planning takes 26 ms
// == Iteration 2 ==
// Query planning takes 53 ms
// == Iteration 3 ==
// Query planning takes 163 ms
// == Iteration 4 ==
// Query planning takes 700 ms
// == Iteration 5 ==
// Query planning takes 3418 ms
```
This is because when building a new Dataset, the new plan is always built upon `QueryExecution.analyzed`, which doesn't leverage existing cached plans.
On the other hand, usually, doing caching every a few iterations may not be the right direction for this problem since caching is too memory consuming (imaging computing connected components over a graph with 50 billion nodes). What we really need here is to truncate both the query plan (to minimize query planning time) and the lineage of the underlying RDD (to avoid stack overflow).
### Changes introduced in this PR
This PR tries to fix this issue by introducing a `checkpoint()` method into `Dataset[T]`, which does exactly the things described above. The following snippet, which is essentially the same as the one above but invokes `checkpoint()` instead of `cache()`, shows the micro benchmark result of this PR:
One key point is that the checkpointed Dataset should preserve the origianl partitioning and ordering information of the original Dataset, so that we can avoid unnecessary shuffling (similar to reading from a pre-bucketed table). This is done by adding `outputPartitioning` and `outputOrdering` to `LogicalRDD` and `RDDScanExec`.
### Micro benchmark
``` scala
spark.sparkContext.setCheckpointDir("/tmp/cp")
(0 until 100).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) =>
println(s"== Iteration $iteration ==")
val time0 = System.currentTimeMillis()
val cp = plan.checkpoint()
cp.count()
System.out.println(s"Checkpointing takes ${System.currentTimeMillis() - time0} ms")
val time1 = System.currentTimeMillis()
val joined = cp.join(cp, "value").join(cp, "value").join(cp, "value").join(cp, "value")
val result = joined.as[Int]
println(s"Query planning takes ${System.currentTimeMillis() - time1} ms")
result
}
// == Iteration 0 ==
// Checkpointing takes 591 ms
// Query planning takes 13 ms
// == Iteration 1 ==
// Checkpointing takes 1605 ms
// Query planning takes 16 ms
// == Iteration 2 ==
// Checkpointing takes 782 ms
// Query planning takes 8 ms
// == Iteration 3 ==
// Checkpointing takes 729 ms
// Query planning takes 10 ms
// == Iteration 4 ==
// Checkpointing takes 734 ms
// Query planning takes 9 ms
// == Iteration 5 ==
// ...
// == Iteration 50 ==
// Checkpointing takes 571 ms
// Query planning takes 7 ms
// == Iteration 51 ==
// Checkpointing takes 548 ms
// Query planning takes 7 ms
// == Iteration 52 ==
// Checkpointing takes 596 ms
// Query planning takes 8 ms
// == Iteration 53 ==
// Checkpointing takes 568 ms
// Query planning takes 7 ms
// ...
```
You may see that although checkpointing is more heavy weight an operation, it always takes roughly the same amount of time to perform both checkpointing and query planning.
### Open question
mengxr mentioned that it would be more convenient if we can make `Dataset.checkpoint()` eager, i.e., always performs a `RDD.count()` after calling `RDD.checkpoint()`. Not quite sure whether this is a universal requirement. Maybe we can add a `eager: Boolean` argument for `Dataset.checkpoint()` to support that.
## How was this patch tested?
Unit test added in `DatasetSuite`.
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#15651 from liancheng/ds-checkpoint.
## What changes were proposed in this pull request?
Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.
This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15663 from zsxwing/fix-event-log.
## What changes were proposed in this pull request?
Currently, `ANALYZE TABLE` command accepts `identifier` for option `NOSCAN`. This PR raises a ParseException for unknown option.
**Before**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []
scala> sql("analyze table test compute statistics blah")
res1: org.apache.spark.sql.DataFrame = []
```
**After**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []
scala> sql("analyze table test compute statistics blah")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected `NOSCAN` instead of `blah`(line 1, pos 0)
```
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15640 from dongjoon-hyun/SPARK-18106.
## What changes were proposed in this pull request?
To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression.
```
TableFileCatalog => CatalogFileIndex
FileCatalog => FileIndex
ListingFileCatalog => InMemoryFileIndex
MetadataLogFileCatalog => MetadataLogFileIndex
PrunedTableFileCatalog => PrunedInMemoryFileIndex
```
cc yhuai marmbrus
## How was this patch tested?
N/A
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#15634 from ericl/rename-file-provider.
## What changes were proposed in this pull request?
Fixed the issue that ForeachSink didn't rethrow the exception.
## How was this patch tested?
The fixed unit test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15674 from zsxwing/foreach-sink-error.
## What changes were proposed in this pull request?
We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:
1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.
## How was this patch tested?
existing tests.
Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15515 from cloud-fan/partition.
## What changes were proposed in this pull request?
A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15661 from zsxwing/fix-StreamingQuerySuite.
## What changes were proposed in this pull request?
This PR is an enhancement of PR with commit ID:57dc326bd00cf0a49da971e9c573c48ae28acaa2.
NaN is a special type of value which is commonly seen as invalid. But We find that there are certain cases where NaN are also valuable, thus need special handling. We provided user when dealing NaN values with 3 options, to either reserve an extra bucket for NaN values, or remove the NaN values, or report an error, by setting handleNaN "keep", "skip", or "error"(default) respectively.
'''Before:
val bucketizer: Bucketizer = new Bucketizer()
.setInputCol("feature")
.setOutputCol("result")
.setSplits(splits)
'''After:
val bucketizer: Bucketizer = new Bucketizer()
.setInputCol("feature")
.setOutputCol("result")
.setSplits(splits)
.setHandleNaN("keep")
## How was this patch tested?
Tests added in QuantileDiscretizerSuite, BucketizerSuite and DataFrameStatSuite
Signed-off-by: VinceShieh <vincent.xieintel.com>
Author: VinceShieh <vincent.xie@intel.com>
Author: Vincent Xie <vincent.xie@intel.com>
Author: Joseph K. Bradley <joseph@databricks.com>
Closes#15428 from VinceShieh/spark-17219_followup.
## What changes were proposed in this pull request?
A short code snippet that uses toLocalIterator() on a dataframe produced by a RunnableCommand
reproduces the problem. toLocalIterator() is called by thriftserver when
`spark.sql.thriftServer.incrementalCollect`is set to handle queries producing large result
set.
**Before**
```SQL
scala> spark.sql("show databases")
res0: org.apache.spark.sql.DataFrame = [databaseName: string]
scala> res0.toLocalIterator()
16/10/26 03:00:24 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
```
**After**
```SQL
scala> spark.sql("drop database databases")
res30: org.apache.spark.sql.DataFrame = []
scala> spark.sql("show databases")
res31: org.apache.spark.sql.DataFrame = [databaseName: string]
scala> res31.toLocalIterator().asScala foreach println
[default]
[parquet]
```
## How was this patch tested?
Added a test in DDLSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#15642 from dilipbiswal/SPARK-18009.
## What changes were proposed in this pull request?
This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.
## How was this patch tested?
Existing regression tests already exercise the new code.
Author: frreiss <frreiss@us.ibm.com>
Closes#14553 from frreiss/fred-16963.
## What changes were proposed in this pull request?
Currently we have several test cases for group analytics(ROLLUP/CUBE/GROUPING SETS) in `SQLQuerySuite`, should better move them into a query file test.
The following test cases are moved to `group-analytics.sql`:
```
test("rollup")
test("grouping sets when aggregate functions containing groupBy columns")
test("cube")
test("grouping sets")
test("grouping and grouping_id")
test("grouping and grouping_id in having")
test("grouping and grouping_id in sort")
```
This is followup work of #15582
## How was this patch tested?
Modified query file `group-analytics.sql`, which will be tested by `SQLQueryTestSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15624 from jiangxb1987/group-analytics-test.
…rdless of warehouse dir's existence
## What changes were proposed in this pull request?
Appending a trailing slash, if there already isn't one for the
sake comparison of the two paths. It doesn't take away from
the essence of the check, but removes any potential mismatch
due to lack of trailing slash.
## How was this patch tested?
Ran unit tests and they passed.
Author: Mark Grover <mark@apache.org>
Closes#15623 from markgrover/spark-18093.
## What changes were proposed in this pull request?
The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.
~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.
## How was this patch tested?
Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15319 from jiangxb1987/constraints.
### What changes were proposed in this pull request?
```SQL
CREATE TABLE tab1(col1 int COMMENT 'a', col2 int) USING parquet
INSERT INTO TABLE tab1 SELECT 1, 2
```
The insert attempt will fail if the target table has a column with comments. The error is strange to the external users:
```
assertion failed: No plan for InsertIntoTable Relation[col1#15,col2#16] parquet, false, false
+- Project [1 AS col1#19, 2 AS col2#20]
+- OneRowRelation$
```
This PR is to fix the above bug by checking the metadata when comparing the schema between the table and the query. If not matched, we also copy the metadata. This is an alternative to https://github.com/apache/spark/pull/15266
### How was this patch tested?
Added a test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15615 from gatorsmile/insertDataSourceTableWithCommentSolution2.
## What changes were proposed in this pull request?
Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array.
## How was this patch tested?
a regression test in `DataFrameSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15606 from cloud-fan/type-bug.
## What changes were proposed in this pull request?
Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.
However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.
This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15566 from cloud-fan/partition-spec.
### What changes were proposed in this pull request?
This follow-up PR is for addressing the [comment](https://github.com/apache/spark/pull/15048).
We added two test cases based on the suggestion from yhuai . One is a new test case using the `saveAsTable` API to create a data source table. Another is for CTAS on Hive serde table.
Note: No need to backport this PR to 2.0. Will submit a new PR to backport the whole fix with new test cases to Spark 2.0
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15459 from gatorsmile/ctasOptimizedTestCases.
## What changes were proposed in this pull request?
The reason for the flakiness was follows. The test starts the maintenance background thread, and then writes 20 versions of the state store. The maintenance thread is expected to create snapshots in the middle, and clean up old files that are not needed any more. The earliest delta file (1.delta) is expected to be deleted as snapshots will ensure that the earliest delta would not be needed.
However, the default configuration for the maintenance thread is to retain files such that last 2 versions can be recovered, and delete the rest. Now while generating the versions, the maintenance thread can kick in and create snapshots anywhere between version 10 and 20 (at least 10 deltas needed for snapshot). Then later it will choose to retain only version 20 and 19 (last 2). There are two cases.
- Common case: One of the version between 10 and 19 gets snapshotted. Then recovering versions 19 and 20 just needs 19.snapshot and 20.delta, so 1.delta gets deleted.
- Uncommon case (reason for flakiness): Only version 20 gets snapshotted. Then recovering versoin 20 requires 20.snapshot, and recovering version 19 all the previous 19...1.delta. So 1.delta does not get deleted.
This PR rearranges the checks such that it create 20 versions, and then waits that there is at least one snapshot, then creates another 20. This will ensure that the latest 2 versions cannot require anything older than the first snapshot generated, and therefore will 1.delta will be deleted.
In addition, I have added more logs, and comments that I felt would help future debugging and understanding what is going on.
## How was this patch tested?
Ran the StateStoreSuite > 6K times in a heavily loaded machine (10 instances of tests running in parallel). No failures.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15592 from tdas/SPARK-17624.
## What changes were proposed in this pull request?
Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#15382 from srowen/SPARK-17810.
## What changes were proposed in this pull request?
The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`.
The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements.
## How was this patch tested?
~~Modified `SQLQuerySuite` in package `sql`.~~
Add query file for `SQLQueryTestSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15582 from jiangxb1987/group-analytics-test.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-18038
This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html
His words:
>> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode.
With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`.
UnaryExecNode's impl | outputPartitioning | outputOrdering | comment
------------ | ------------- | ------------ | ------------
AppendColumnsExec | child's | Nil | child's ordering can be used
AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used
BroadcastExchangeExec | BroadcastPartitioning | Nil | -
CoalesceExec | UnknownPartitioning | Nil | -
CollectLimitExec | SinglePartition | Nil | -
DebugExec | child's | Nil | child's ordering can be used
DeserializeToObjectExec | child's | Nil | child's ordering can be used
ExpandExec | UnknownPartitioning | Nil | -
FilterExec | child's | child's | -
FlatMapGroupsInRExec | child's | Nil | child's ordering can be used
GenerateExec | child's | Nil | need to dig more
GlobalLimitExec | child's | child's | -
HashAggregateExec | child's | Nil | -
InputAdapter | child's | child's | -
InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning
LocalLimitExec | child's | child's | -
MapElementsExec | child's | child's | -
MapGroupsExec | child's | Nil | child's ordering can be used
MapPartitionsExec | child's | Nil | child's ordering can be used
ProjectExec | child's | child's | -
SampleExec | child's | Nil | child's ordering can be used
ScriptTransformation | child's | Nil | child's ordering can be used
SerializeFromObjectExec | child's | Nil | child's ordering can be used
ShuffleExchange | custom | Nil | -
SortAggregateExec | child's | sort over grouped exprs | -
SortExec | child's | custom | -
StateStoreRestoreExec | child's | Nil | child's ordering can be used
StateStoreSaveExec | child's | Nil | child's ordering can be used
SubqueryExec | child's | child's | -
TakeOrderedAndProjectExec | SinglePartition | custom | -
WholeStageCodegenExec | child's | child's | -
WindowExec | child's | child's | -
## How was this patch tested?
This does NOT change any existing functionality so relying on existing tests
Author: Tejas Patil <tejasp@fb.com>
Closes#15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning.
## What changes were proposed in this pull request?
In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again.
This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15581 from cloud-fan/stream.
## What changes were proposed in this pull request?
StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`.
## How was this patch tested?
New unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15476 from tdas/SPARK-17926.
## What changes were proposed in this pull request?
`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.
cc srowen
## How was this patch tested?
existing tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#15564 from zhengruifeng/avoid_0_length_array.
## What changes were proposed in this pull request?
My hunch is `mkdirs` fails. Just add more checks to collect more info.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15577 from zsxwing/SPARK-18030-debug.
## What changes were proposed in this pull request?
Add mapValues to KeyValueGroupedDataset
## How was this patch tested?
New test in DatasetSuite for groupBy function, mapValues, flatMap
Author: Koert Kuipers <koert@tresata.com>
Closes#13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
## What changes were proposed in this pull request?
SHOW COLUMNS command validates the user supplied database
name with database name from qualified table name name to make
sure both of them are consistent. This comparison should respect
case sensitivity.
## How was this patch tested?
Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure.
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#15423 from dilipbiswal/dkb_show_column_fix.
## What changes were proposed in this pull request?
This PR proposes to check the second argument, `ascendingOrder` rather than throwing `ClassCastException` exception message.
```sql
select sort_array(array('b', 'd'), '1');
```
**Before**
```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```
**After**
```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```
## How was this patch tested?
Unit test in `DataFrameFunctionsSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15532 from HyukjinKwon/SPARK-17989.
## What changes were proposed in this pull request?
Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.
However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.
This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.
## How was this patch tested?
The added back tests and some new tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15434 from cloud-fan/revert.
## What changes were proposed in this pull request?
As per rxin request, here are further API changes
- Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event`
- Changed the fields in `StreamingQueryListener.on***` from `query*` to `event`
## How was this patch tested?
Existing unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15530 from tdas/SPARK-17731-1.
## What changes were proposed in this pull request?
This renames `BasicFileCatalog => FileCatalog`, combines `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.
In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```
(note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted)
## How was this patch tested?
Existing tests
Author: Eric Liang <ekl@databricks.com>
Closes#15533 from ericl/fix-scalastyle-revert.
## What changes were proposed in this pull request?
Currently, Spark only supports to infer `IntegerType`, `LongType`, `DoubleType` and `StringType`.
`DecimalType` is being tried but it seems it never infers type as `DecimalType` as `DoubleType` is being tried first. Also, it seems `DateType` and `TimestampType` could be inferred.
As far as I know, it is pretty common to use both for a partition column.
This PR fixes the incorrect `DecimalType` try and also adds the support for both `DateType` and `TimestampType` for inferring partition column type.
## How was this patch tested?
Unit tests in `ParquetPartitionDiscoverySuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14947 from HyukjinKwon/SPARK-17388.
## What changes were proposed in this pull request?
Scala 2.10 does not have Option.contains, which broke Scala 2.10 build.
## How was this patch tested?
Locally compiled and ran sql/core unit tests in 2.10
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15531 from tdas/metrics-flaky-test-fix-1.
This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it.
## What changes were proposed in this pull request?
There were two sources of flakiness in StreamingQueryListener test.
- When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock.
```
+-----------------------------------+--------------------------------+
| StreamExecution thread | testing thread |
+-----------------------------------+--------------------------------+
| ManualClock.waitTillTime(100) { | |
| _isWaiting = true | |
| wait(10) | |
| still in wait(10) | if (_isWaiting) advance(100) |
| still in wait(10) | if (_isWaiting) advance(200) | <- this should be disallowed !
| still in wait(10) | if (_isWaiting) advance(300) | <- this should be disallowed !
| wake up from wait(10) | |
| current time is 600 | |
| _isWaiting = false | |
| } | |
+-----------------------------------+--------------------------------+
```
- Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger.
My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`).
In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest.
## How was this patch tested?
Ran existing unit test MANY TIME in Jenkins
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Liwei Lin <lwlin7@gmail.com>
Closes#15519 from tdas/metrics-flaky-test-fix.
## What changes were proposed in this pull request?
This renames `BasicFileCatalog => FileCatalog`, combines `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.
In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```
cc cloud-fan mallman
## How was this patch tested?
Existing tests
Author: Eric Liang <ekl@databricks.com>
Closes#15518 from ericl/refactor-session-file-catalog.
### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.
This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```
After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
+- Project [_1#2 AS k#5, _2#3 AS v#6]
+- LocalRelation [_1#2, _2#3]
```
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15316 from gatorsmile/eagerAnalysis.
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)
## What changes were proposed in this pull request?
In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.
If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.
In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.
This PR proposes an alternative approach. Basically, it makes four changes:
1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.
The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.
As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.
## Open Issues
1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.
## How was this patch tested?
The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#14690 from mallman/spark-16980-lazy_partition_fetching.
Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits:
* Leverage the power of rich third party java library
* Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance.
Author: Jeff Zhang <zjffdu@apache.org>
Closes#9766 from zjffdu/SPARK-11775.
[SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`.
Updated `DatasetCacheSuite`.
Author: Nick Pentreath <nickp@za.ibm.com>
Closes#13780 from MLnick/ds-storagelevel.
Signed-off-by: Michael Armbrust <michael@databricks.com>
## What changes were proposed in this pull request?
We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that.
## How was this patch tested?
Added regression test.
Author: Davies Liu <davies@databricks.com>
Closes#15489 from davies/order_distinct.
## What changes were proposed in this pull request?
There are 4 listLeafFiles-related functions in Spark:
- ListingFileCatalog.listLeafFiles (which calls HadoopFsRelation.listLeafFilesInParallel if the number of paths passed in is greater than a threshold; if it is lower, then it has its own serial version implemented)
- HadoopFsRelation.listLeafFiles (called only by HadoopFsRelation.listLeafFilesInParallel)
- HadoopFsRelation.listLeafFilesInParallel (called only by ListingFileCatalog.listLeafFiles)
It is actually very confusing and error prone because there are effectively two distinct implementations for the serial version of listing leaf files. As an example, SPARK-17599 updated only one of the code path and ignored the other one.
This code can be improved by:
- Move all file listing code into ListingFileCatalog, since it is the only class that needs this.
- Keep only one function for listing files in serial.
## How was this patch tested?
This change should be covered by existing unit and integration tests. I also moved a test case for HadoopFsRelation.shouldFilterOut from HadoopFsRelationSuite to ListingFileCatalogSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#15235 from petermaxlee/SPARK-17661.
## What changes were proposed in this pull request?
Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing
Specifically, this PR adds the following public APIs changes.
### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)
- `StreamingQueryStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by all the sources
- processingRate - Current rate (rows/sec) at which the query is processing data from
all the sources
- ~~outputRate~~ - *Does not work with wholestage codegen*
- latency - Current average latency between the data being available in source and the sink writing the corresponding output
- sourceStatuses: Array[SourceStatus] - Current statuses of the sources
- sinkStatus: SinkStatus - Current status of the sink
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- latencies - getOffset, getBatch, full trigger, wal writes
- timestamps - trigger start, finish, after getOffset, after getBatch
- numRows - input, output, state total/updated rows for aggregations
- `SourceStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by the source
- processingRate - Current rate (rows/sec) at which the query is processing data from the source
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- Python API for `StreamingQuery.status()`
### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
- Branch 2.0 should have it deprecated, master should have it removed.
**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
- Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)
- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.
- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.
- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.
## How was this patch tested?
Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.
Metrics also manually tested using Ganglia sink
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15307 from tdas/SPARK-17731.
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int
## How was this patch tested?
Test runs on little endian and big endian platforms
Author: Pete Robbins <robbinspg@gmail.com>
Closes#15464 from robbinspg/SPARK-17827.
## What changes were proposed in this pull request?
Currently `HiveExternalCatalog` will filter out the Spark SQL internal table properties, e.g. `spark.sql.sources.provider`, `spark.sql.sources.schema`, etc. This is reasonable for external users as they don't want to see these internal properties in `DESC TABLE`.
However, as a Spark developer, sometimes we do wanna see the raw table properties. This PR adds a new internal SQL conf, `spark.sql.debug`, to enable debug mode and keep these raw table properties.
This config can also be used in similar places where we wanna retain debug information in the future.
## How was this patch tested?
new test in MetastoreDataSourcesSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15458 from cloud-fan/debug.
## What changes were proposed in this pull request?
Two issues regarding Dataset.dropduplicates:
1. Dataset.dropDuplicates should consider the columns with same column name
We find and get the first resolved attribute from output with the given column name in `Dataset.dropDuplicates`. When we have the more than one columns with the same name. Other columns are put into aggregation columns, instead of grouping columns.
2. Dataset.dropDuplicates should not change the output of child plan
We create new `Alias` with new exprId in `Dataset.dropDuplicates` now. However it causes problem when we want to select the columns as follows:
val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS()
// ds("_2") will cause analysis exception
ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int])
Because the two issues are both related to `Dataset.dropduplicates` and the code changes are not big, so submitting them together as one PR.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#15427 from viirya/fix-dropduplicates.
## What changes were proposed in this pull request?
The CompactibleFileStreamLog materializes the whole metadata log in memory as a String. This can cause issues when there are lots of files that are being committed, especially during a compaction batch.
You may come across stacktraces that look like:
```
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
at java.lang.StringCoding.encode(StringCoding.java:350)
at java.lang.String.getBytes(String.java:941)
at org.apache.spark.sql.execution.streaming.FileStreamSinkLog.serialize(FileStreamSinkLog.scala:127)
```
The safer way is to write to an output stream so that we don't have to materialize a huge string.
## How was this patch tested?
Existing unit tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15437 from brkyvz/ser-to-stream.
## What changes were proposed in this pull request?
This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are:
1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding).
2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense.
Code example required to specify a frame before this patch:
```
Window.rowsBetween(-Long.MinValue, 0)
```
While the above code should still work, the new way is more obvious to read:
```
Window.rowsBetween(Window.unboundedPreceding, Window.currentRow)
```
## How was this patch tested?
- Updated DataFrameWindowSuite (for Scala/Java)
- Updated test_window_functions_cumulative_sum (for Python)
- Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose
Author: Reynold Xin <rxin@databricks.com>
Closes#15438 from rxin/SPARK-17845.
## What changes were proposed in this pull request?
This is a step along the way to SPARK-8425.
To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)
Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.
Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).
## How was this patch tested?
Added unit tests, run tests via jenkins.
Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>
Closes#15249 from squito/taskset_blacklist_only.
## What changes were proposed in this pull request?
Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`.
## How was this patch tested?
The added unit tests
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15422 from zsxwing/SPARK-17850.
## What changes were proposed in this pull request?
SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.
Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.
## How was this patch tested?
new tests in SQLConfSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15295 from cloud-fan/global-conf.
## What changes were proposed in this pull request?
When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object.
As an example, to compute cumulative sum using the natural ordering, before this pr:
```
df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0)))
```
After this pr:
```
df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0)))
```
Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available.
This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala).
## How was this patch tested?
Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python.
Author: Reynold Xin <rxin@databricks.com>
Closes#15412 from rxin/SPARK-17844.
## What changes were proposed in this pull request?
This PR proposes to fix arbitrary usages among `Map[String, String]`, `Properties` and `JDBCOptions` instances for options in `execution/jdbc` package and make the connection properties exclude Spark-only options.
This PR includes some changes as below:
- Unify `Map[String, String]`, `Properties` and `JDBCOptions` in `execution/jdbc` package to `JDBCOptions`.
- Move `batchsize`, `fetchszie`, `driver` and `isolationlevel` options into `JDBCOptions` instance.
- Document `batchSize` and `isolationlevel` with marking both read-only options and write-only options. Also, this includes minor types and detailed explanation for some statements such as url.
- Throw exceptions fast by checking arguments first rather than in execution time (e.g. for `fetchsize`).
- Exclude Spark-only options in connection properties.
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15292 from HyukjinKwon/SPARK-17719.
## What changes were proposed in this pull request?
Currently, CSV datasource allows to load duplicated empty string fields or fields having `nullValue` in the header. It'd be great if this can deal with normal fields as well.
This PR proposes handling the duplicates consistently with the existing behaviour with considering case-sensitivity (`spark.sql.caseSensitive`) as below:
data below:
```
fieldA,fieldB,,FIELDA,fielda,,
1,2,3,4,5,6,7
```
is parsed as below:
```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```
- when `spark.sql.caseSensitive` is `false` (by default).
```
+-------+------+---+-------+-------+---+---+
|fieldA0|fieldB|_c2|FIELDA3|fieldA4|_c5|_c6|
+-------+------+---+-------+-------+---+---+
| 1| 2| 3| 4| 5| 6| 7|
+-------+------+---+-------+-------+---+---+
```
- when `spark.sql.caseSensitive` is `true`.
```
+-------+------+---+-------+-------+---+---+
|fieldA0|fieldB|_c2| FIELDA|fieldA4|_c5|_c6|
+-------+------+---+-------+-------+---+---+
| 1| 2| 3| 4| 5| 6| 7|
+-------+------+---+-------+-------+---+---+
```
**In more details**,
There is a good reference about this problem, `read.csv()` in R. So, I initially wanted to propose the similar behaviour.
In case of R, the CSV data below:
```
fieldA,fieldB,,fieldA,fieldA,,
1,2,3,4,5,6,7
```
is parsed as below:
```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
fieldA fieldB X fieldA.1 fieldA.2 X.1 X.2
1 1 2 3 4 5 6 7
```
However, Spark CSV datasource already is handling duplicated empty strings and `nullValue` as field names. So the data below:
```
,,,fieldA,,fieldB,
1,2,3,4,5,6,7
```
is parsed as below:
```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```
```
+---+---+---+------+---+------+---+
|_c0|_c1|_c2|fieldA|_c4|fieldB|_c6|
+---+---+---+------+---+------+---+
| 1| 2| 3| 4| 5| 6| 7|
+---+---+---+------+---+------+---+
```
R starts the number for each duplicate but Spark adds the number for its position for all fields for `nullValue` and empty strings.
In terms of case-sensitivity, it seems R is case-sensitive as below: (it seems it is not configurable).
```
a,a,a,A,A
1,2,3,4,5
```
is parsed as below:
```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
a a.1 a.2 A A.1
1 1 2 3 4 5
```
## How was this patch tested?
Unit test in `CSVSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14745 from HyukjinKwon/SPARK-16896.
## What changes were proposed in this pull request?
The default buffer size is not big enough for randomly generated MapType.
## How was this patch tested?
Ran the tests in 100 times, it never fail (it fail 8 times before the patch).
Author: Davies Liu <davies@databricks.com>
Closes#15395 from davies/flaky_map.
## What changes were proposed in this pull request?
The function `SparkSqlParserSuite.createTempViewUsing` is not used for now and causes build failure, this PR simply removes it.
## How was this patch tested?
N/A
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15418 from jiangxb1987/parserSuite.
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14897 from cloud-fan/global-temp-view.
## What changes were proposed in this pull request?
Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
id bigint,
nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.
Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```
## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15346 from jiangxb1987/cdt.
## What changes were proposed in this pull request?
In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes.
## How was this patch tested?
Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug.
Author: Davies Liu <davies@databricks.com>
Closes#15390 from davies/rewrite_key.
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.
The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```
This might be somewhat controversial, so feedback is appreciated.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15333 from hvanhovell/SPARK-17761.
## What changes were proposed in this pull request?
When execute a Python UDF, we buffer the input row into as queue, then pull them out to join with the result from Python UDF. In the case that Python UDF is slow or the input row is too wide, we could ran out of memory because of the queue. Since we can't flush all the buffers (sockets) between JVM and Python process from JVM side, we can't limit the rows in the queue, otherwise it could deadlock.
This PR will manage the memory used by the queue, spill that into disk when there is no enough memory (also release the memory and disk space as soon as possible).
## How was this patch tested?
Added unit tests. Also manually ran a workload with large input row and slow python UDF (with large broadcast) like this:
```
b = range(1<<24)
add = udf(lambda x: x + len(b), IntegerType())
df = sqlContext.range(1, 1<<26, 1, 4)
print df.select(df.id, lit("adf"*10000).alias("s"), add(df.id).alias("add")).groupBy(length("s")).sum().collect()
```
It ran out of memory (hang because of full GC) before the patch, ran smoothly after the patch.
Author: Davies Liu <davies@databricks.com>
Closes#15089 from davies/spill_udf.
## What changes were proposed in this pull request?
Adds the textFile API which exists in DataFrameReader and serves same purpose.
## How was this patch tested?
Added corresponding testcase.
Author: Prashant Sharma <prashsh1@in.ibm.com>
Closes#14087 from ScrapCodes/textFile.
## What changes were proposed in this pull request?
This PR proposes cleaning up the confusing part in `createRelation` as discussed in https://github.com/apache/spark/pull/12601/files#r80627940
Also, this PR proposes the changes below:
- Add documentation for `batchsize` and `isolationLevel`.
- Move property names into `JDBCOptions` so that they can be managed in a single place. which were, `fetchsize`, `batchsize`, `isolationLevel` and `driver`.
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15263 from HyukjinKwon/SPARK-14525.
## What changes were proposed in this pull request?
When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying.
## How was this patch tested?
`test("NoClassDefFoundError from an incompatible source")`
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15352 from zsxwing/SPARK-17780.
## What changes were proposed in this pull request?
This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source.
It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing
tdas did most of work and part of them was inspired by koeninger's work.
### Introduction
The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows:
Column | Type
---- | ----
key | binary
value | binary
topic | string
partition | int
offset | long
timestamp | long
timestampType | int
The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic.
### Configuration
The user can use `DataStreamReader.option` to set the following configurations.
Kafka Source's options | value | default | meaning
------ | ------- | ------ | -----
startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off.
failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected.
subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors
fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets
Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")`
### Usage
* Subscribe to 1 topic
```Scala
spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host:port")
.option("subscribe", "topic1")
.load()
```
* Subscribe to multiple topics
```Scala
spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host:port")
.option("subscribe", "topic1,topic2")
.load()
```
* Subscribe to a pattern
```Scala
spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host:port")
.option("subscribePattern", "topic.*")
.load()
```
## How was this patch tested?
The new unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: cody koeninger <cody@koeninger.org>
Closes#15102 from zsxwing/kafka-source.
## What changes were proposed in this pull request?
This PR fixes the following NPE scenario in two ways.
**Reported Error Scenario**
```scala
scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false)
INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x
java.lang.NullPointerException
```
- **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`.
- **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`.
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15357 from dongjoon-hyun/SPARK-17328.
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.
This implications in tests are limited to a single Hive compatibility test.
## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14828 from hvanhovell/SPARK-17258.
This reverts commit 9ac68dbc57. Turns out
the original fix was correct.
Original change description:
The existing code caches all stats for all columns for each partition
in the driver; for a large relation, this causes extreme memory usage,
which leads to gc hell and application failures.
It seems that only the size in bytes of the data is actually used in the
driver, so instead just colllect that. In executors, the full stats are
still kept, but that's not a big problem; we expect the data to be distributed
and thus not really incur in too much memory pressure in each individual
executor.
There are also potential improvements on the executor side, since the data
being stored currently is very wasteful (e.g. storing boxed types vs.
primitive types for stats). But that's a separate issue.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#15304 from vanzin/SPARK-17549.2.
## What changes were proposed in this pull request?
Generate basic column statistics for all the atomic types:
- numeric types: max, min, num of nulls, ndv (number of distinct values)
- date/timestamp types: they are also represented as numbers internally, so they have the same stats as above.
- string: avg length, max length, num of nulls, ndv
- binary: avg length, max length, num of nulls
- boolean: num of nulls, num of trues, num of falsies
Also support storing and loading these statistics.
One thing to notice:
We support analyzing columns independently, e.g.:
sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;`
sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;`
when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`:
`ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;`
## How was this patch tested?
add unit tests
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#15090 from wzhfy/colStats.
## What changes were proposed in this pull request?
This PR proposes to fix/skip some tests failed on Windows. This PR takes over https://github.com/apache/spark/pull/12696.
**Before**
- **SparkSubmitSuite**
```
[info] - launch simple application with spark-submit *** FAILED *** (202 milliseconds)
[info] java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specifie
[info] - includes jars passed in through --jars *** FAILED *** (1 second, 625 milliseconds)
[info] java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```
- **DiskStoreSuite**
```
[info] - reads of memory-mapped and non memory-mapped files are equivalent *** FAILED *** (1 second, 78 milliseconds)
[info] diskStoreMapped.remove(blockId) was false (DiskStoreSuite.scala:41)
```
**After**
- **SparkSubmitSuite**
```
[info] - launch simple application with spark-submit (578 milliseconds)
[info] - includes jars passed in through --jars (1 second, 875 milliseconds)
```
- **DiskStoreSuite**
```
[info] DiskStoreSuite:
[info] - reads of memory-mapped and non memory-mapped files are equivalent !!! CANCELED !!! (766 milliseconds
```
For `CreateTableAsSelectSuite` and `FsHistoryProviderSuite`, I could not reproduce as the Java version seems higher than the one that has the bugs about `setReadable(..)` and `setWritable(...)` but as they are bugs reported clearly, it'd be sensible to skip those. We should revert the changes for both back as soon as we drop the support of Java 7.
## How was this patch tested?
Manually tested via AppVeyor.
Closes#12696
Author: Tao LI <tl@microsoft.com>
Author: U-FAREAST\tl <tl@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15320 from HyukjinKwon/SPARK-14914.
## What changes were proposed in this pull request?
We added find and exists methods for Databases, Tables and Functions to the user facing Catalog in PR https://github.com/apache/spark/pull/15301. However, it was brought up that the semantics of the `find` methods are more in line a `get` method (get an object or else fail). So we rename these in this PR.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15308 from hvanhovell/SPARK-17717-2.
## What changes were proposed in this pull request?
As a followup to SPARK-17666, ensure filesystem connections are not leaked at least in unit tests. This is done here by intercepting filesystem calls as suggested by JoshRosen . At the end of each test, we assert no filesystem streams are left open.
This applies to all tests using SharedSQLContext or SharedSparkContext.
## How was this patch tested?
I verified that tests in sql and core are indeed using the filesystem backend, and fixed the detected leaks. I also checked that reverting https://github.com/apache/spark/pull/15245 causes many actual test failures due to connection leaks.
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#15306 from ericl/sc-4672.
## What changes were proposed in this pull request?
The actualSize() of array and map is different from the actual size, the header is Int, rather than Long.
## How was this patch tested?
The flaky test should be fixed.
Author: Davies Liu <davies@databricks.com>
Closes#15305 from davies/fix_MAP.
## What changes were proposed in this pull request?
The current user facing catalog does not implement methods for checking object existence or finding objects. You could theoretically do this using the `list*` commands, but this is rather cumbersome and can actually be costly when there are many objects. This PR adds `exists*` and `find*` methods for Databases, Table and Functions.
## How was this patch tested?
Added tests to `org.apache.spark.sql.internal.CatalogSuite`
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15301 from hvanhovell/SPARK-17717.
Spark SQL has great support for reading text files that contain JSON data. However, in many cases the JSON data is just one column amongst others. This is particularly true when reading from sources such as Kafka. This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema.
Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)
df.select(from_json($"value", schema) as 'json) // => [json: <a: int>]
```
This PR adds support for java, scala and python. I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it). I left SQL out for now, because I'm not sure how users would specify a schema.
Author: Michael Armbrust <michael@databricks.com>
Closes#15274 from marmbrus/jsonParser.
## What changes were proposed in this pull request?
We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method.
## How was this patch tested?
Added a regression test to `DataFrameAggregateSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15208 from hvanhovell/SPARK-17641.
## What changes were proposed in this pull request?
As a followup for https://github.com/apache/spark/pull/15273 we should move non-JDBC specific tests out of that suite.
## How was this patch tested?
Ran the test.
Author: Eric Liang <ekl@databricks.com>
Closes#15287 from ericl/spark-17713.
## What changes were proposed in this pull request?
It seems the equality check for reuse of `RowDataSourceScanExec` nodes doesn't respect the output schema. This can cause self-joins or unions over the same underlying data source to return incorrect results if they select different fields.
## How was this patch tested?
New unit test passes after the fix.
Author: Eric Liang <ekl@databricks.com>
Closes#15273 from ericl/spark-17673.
## What changes were proposed in this pull request?
This PR introduces more compact representation for ```UnsafeArrayData```.
```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts
```
[numElements] [offsets] [values]
```
`Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`.
This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts.
```
[numElements][null bits][values or offset&length][variable length portion]
```
In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries.
In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries.
The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison:
1024x1024 elements integer array
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes
In summary, we got 1.0-2.6x performance improvements over the code before applying this PR.
Here are performance results of [benchmark programs](04d2e4b6db/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala):
**Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 430 / 436 390.0 2.6 1.0X
Double 456 / 485 367.8 2.7 0.9X
With SPARK-15962
Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 252 / 260 666.1 1.5 1.0X
Double 281 / 292 597.7 1.7 0.9X
````
**Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 203 / 273 103.4 9.7 1.0X
Double 239 / 356 87.9 11.4 0.8X
With SPARK-15962
Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 196 / 249 107.0 9.3 1.0X
Double 227 / 367 92.3 10.8 0.9X
````
**Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 207 / 217 304.2 3.3 1.0X
Double 257 / 363 245.2 4.1 0.8X
With SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 151 / 198 415.8 2.4 1.0X
Double 214 / 394 293.6 3.4 0.7X
````
**Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 340 / 385 185.1 5.4 1.0X
Double 479 / 705 131.3 7.6 0.7X
With SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 206 / 211 306.0 3.3 1.0X
Double 232 / 406 271.6 3.7 0.9X
````
1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala) over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
serialize 442 / 533 0.0 441927.1 1.0X
deserialize 217 / 274 0.0 217087.6 2.0X
With SPARK-15962
VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
serialize 265 / 318 0.0 265138.5 1.0X
deserialize 155 / 197 0.0 154611.4 1.7X
````
## How was this patch tested?
Added unit tests into ```UnsafeArraySuite```
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#13680 from kiszk/SPARK-15962.
## What changes were proposed in this pull request?
This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader.
## How was this patch tested?
Exisiting Unit Tests
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#15225 from sameeragarwal/error-msg.
## What changes were proposed in this pull request?
When reading file stream with non-globbing path, the results return data with all `null`s for the
partitioned columns. E.g.,
case class A(id: Int, value: Int)
val data = spark.createDataset(Seq(
A(1, 1),
A(2, 2),
A(2, 3))
)
val url = "/tmp/test"
data.write.partitionBy("id").parquet(url)
spark.read.parquet(url).show
+-----+---+
|value| id|
+-----+---+
| 2| 2|
| 3| 2|
| 1| 1|
+-----+---+
val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url)
s.writeStream.queryName("test").format("memory").start()
sql("SELECT * FROM test").show
+-----+----+
|value| id|
+-----+----+
| 2|null|
| 3|null|
| 1|null|
+-----+----+
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#14803 from viirya/filestreamsource-option.
## What changes were proposed in this pull request?
This change modifies the implementation of DataFrameWriter.save such that it works with jdbc, and the call to jdbc merely delegates to save.
## How was this patch tested?
This was tested via unit tests in the JDBCWriteSuite, of which I added one new test to cover this scenario.
## Additional details
rxin This seems to have been most recently touched by you and was also commented on in the JIRA.
This contribution is my original work and I license the work to the project under the project's open source license.
Author: Justin Pihony <justin.pihony@gmail.com>
Author: Justin Pihony <justin.pihony@typesafe.com>
Closes#12601 from JustinPihony/jdbc_reconciliation.
## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).
Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.
## How was this patch tested?
Added a new test case in DataFrameSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Author: Xin Wu <xinwu@us.ibm.com>
Closes#15123 from petermaxlee/SPARK-17551.
For some sources, it is difficult to provide a global ordering based only on the data in the offset. Since we don't use comparison for correctness, lets remove it.
Author: Michael Armbrust <michael@databricks.com>
Closes#15207 from marmbrus/removeComparable.
## What changes were proposed in this pull request?
Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example).
## How was this patch tested?
Jenkins.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#15206 from zsxwing/cleanup.
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select grp,
collect_list(col1),
count(distinct col2)
from tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).
## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15187 from hvanhovell/SPARK-17616.
## What changes were proposed in this pull request?
Consider you have a bucket as `s3a://some-bucket`
and under it you have files:
```
s3a://some-bucket/file1.parquet
s3a://some-bucket/file2.parquet
```
Getting the parent path of `s3a://some-bucket/file1.parquet` yields
`s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map.
When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list!
This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics.
## How was this patch tested?
Unit test in `FileCatalogSuite`.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15169 from brkyvz/SPARK-17613.
## What changes were proposed in this pull request?
We should set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation, otherwise the outputs of LogicalRelation are different from outputs of SimpleCatalogRelation - they have different exprId's.
## How was this patch tested?
add a test case
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#15182 from wzhfy/expectedAttributes.
### What changes were proposed in this pull request?
For data sources without extending `SchemaRelationProvider`, we expect users to not specify schemas when they creating tables. If the schema is input from users, an exception is issued.
Since Spark 2.1, for any data source, to avoid infer the schema every time, we store the schema in the metastore catalog. Thus, when reading a cataloged data source table, the schema could be read from metastore catalog. In this case, we also got an exception. For example,
```Scala
sql(
s"""
|CREATE TABLE relationProvierWithSchema
|USING org.apache.spark.sql.sources.SimpleScanSource
|OPTIONS (
| From '1',
| To '10'
|)
""".stripMargin)
spark.table(tableName).show()
```
```
org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas.;
```
This PR is to fix the above issue. When building a data source, we introduce a flag `isSchemaFromUsers` to indicate whether the schema is really input from users. If true, we issue an exception. Otherwise, we will call the `createRelation` of `RelationProvider` to generate the `BaseRelation`, in which it contains the actual schema.
### How was this patch tested?
Added a few cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15046 from gatorsmile/tempViewCases.
## What changes were proposed in this pull request?
While getting the batch for a `FileStreamSource` in StructuredStreaming, we know which files we must take specifically. We already have verified that they exist, and have committed them to a metadata log. When creating the FileSourceRelation however for an incremental execution, the code checks the existence of every single file once again!
When you have 100,000s of files in a folder, creating the first batch takes 2 hours+ when working with S3! This PR disables that check
## How was this patch tested?
Added a unit test to `FileStreamSource`.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15122 from brkyvz/SPARK-17569.
## What changes were proposed in this pull request?
We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query.
By analyzing CTE definitions before substitution, we can support defining CTE in subquery.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#15146 from viirya/cte-analysis-once.
## What changes were proposed in this pull request?
The `ListingFileCatalog` lists files given a set of resolved paths. If a folder is deleted at any time between the paths were resolved and the file catalog can check for the folder, the Spark job fails. This may abruptly stop long running StructuredStreaming jobs for example.
Folders may be deleted by users or automatically by retention policies. These cases should not prevent jobs from successfully completing.
## How was this patch tested?
Unit test in `FileCatalogSuite`
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15153 from brkyvz/SPARK-17599.
## What changes were proposed in this pull request?
This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235.
This is a resubmission of 15126, which was based on work by frreiss in #15067, but fixed the test case along with some typos.
## How was this patch tested?
A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#15166 from petermaxlee/SPARK-17513-2.
## What changes were proposed in this pull request?
Current `metadataLog` in `FileStreamSource` will add a checkpoint file in each batch but do not have the ability to remove/compact, which will lead to large number of small files when running for a long time. So here propose to compact the old logs into one file. This method is quite similar to `FileStreamSinkLog` but simpler.
## How was this patch tested?
Unit test added.
Author: jerryshao <sshao@hortonworks.com>
Closes#13513 from jerryshao/SPARK-15698.
### What changes were proposed in this pull request?
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example,
```
Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`';
```
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example,
```
Attempted to unset non-existent property 'p' in table '`testView`';
```
- When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error:
```
ANALYZE TABLE is not supported for Project
```
- When inserting into a temporary view that is generated from `Range`, we will get the following error message:
```
assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false
+- Project [1 AS 1#20]
+- OneRowRelation$
```
This PR is to fix the above four issues.
### How was this patch tested?
Added multiple test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15054 from gatorsmile/tempViewDDL.
## What changes were proposed in this pull request?
This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235.
This is based on work by frreiss in #15067, but fixed the test case along with some typos.
## How was this patch tested?
A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request.
Author: petermaxlee <petermaxlee@gmail.com>
Author: frreiss <frreiss@us.ibm.com>
Closes#15126 from petermaxlee/SPARK-17513.
## What changes were proposed in this pull request?
AssertOnQuery has two apply constructor: one that accepts a closure that returns boolean, and another that accepts a closure that returns Unit. This is actually very confusing because developers could mistakenly think that AssertOnQuery always require a boolean return type and verifies the return result, when indeed the value of the last statement is ignored in one of the constructors.
This pull request makes the two constructor consistent and always require boolean value. It will overall make the test suites more robust against developer errors.
As an evidence for the confusing behavior, this change also identified a bug with an existing test case due to file system time granularity. This pull request fixes that test case as well.
## How was this patch tested?
This is a test only change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#15127 from petermaxlee/SPARK-17571.
## Problem
CSV in Spark 2.0.0:
- does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6;
- does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903.
## What changes were proposed in this pull request?
This patch makes changes to read all empty values back as `null`s.
## How was this patch tested?
New test cases.
Author: Liwei Lin <lwlin7@gmail.com>
Closes#14118 from lw-lin/csv-cast-null.
## What changes were proposed in this pull request?
In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are:
1. `CREATE TABLE USING` will fail if a same-name temp view exists
2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists
3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists.
These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15099 from cloud-fan/fix-view.
### What changes were proposed in this pull request?
In Spark 2.1, we introduced a new internal provider `hive` for telling Hive serde tables from data source tables. This PR is to block users to specify this in `DataFrameWriter` and SQL APIs.
### How was this patch tested?
Added a test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15073 from gatorsmile/formatHive.
The existing code caches all stats for all columns for each partition
in the driver; for a large relation, this causes extreme memory usage,
which leads to gc hell and application failures.
It seems that only the size in bytes of the data is actually used in the
driver, so instead just colllect that. In executors, the full stats are
still kept, but that's not a big problem; we expect the data to be distributed
and thus not really incur in too much memory pressure in each individual
executor.
There are also potential improvements on the executor side, since the data
being stored currently is very wasteful (e.g. storing boxed types vs.
primitive types for stats). But that's a separate issue.
On a mildly related change, I'm also adding code to catch exceptions in the
code generator since Janino was breaking with the test data I tried this
patch on.
Tested with unit tests and by doing a count a very wide table (20k columns)
with many partitions.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#15112 from vanzin/SPARK-17549.
## What changes were proposed in this pull request?
This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like:
1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error.
2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error.
For a code example, please check the Jira description of SPARK-17426.
In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type.
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14990 from clockfly/json_oom2.
## What changes were proposed in this pull request?
This change preserves aliases that are given for pivot aggregations
## How was this patch tested?
New unit test
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#15111 from aray/SPARK-17458.
## What changes were proposed in this pull request?
select length(11);
select length(2.0);
these sql will return errors, but hive is ok.
this PR will support casting input types implicitly for function length
the correct result is:
select length(11) return 2
select length(2.0) return 3
Author: 岑玉海 <261810726@qq.com>
Author: cenyuhai <cenyuhai@didichuxing.com>
Closes#15014 from cenyuhai/SPARK-17429.
## What changes were proposed in this pull request?
This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row).
This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys.
## How was this patch tested?
Added tests to `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15101 from hvanhovell/SPARK-17114-3.
### What changes were proposed in this pull request?
For the following `ALTER TABLE` DDL, we should issue an exception when the target table is a `VIEW`:
```SQL
ALTER TABLE viewName SET LOCATION '/path/to/your/lovely/heart'
ALTER TABLE viewName SET SERDE 'whatever'
ALTER TABLE viewName SET SERDEPROPERTIES ('x' = 'y')
ALTER TABLE viewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')
ALTER TABLE viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')
ALTER TABLE viewName DROP IF EXISTS PARTITION (a='2')
ALTER TABLE viewName RECOVER PARTITIONS
ALTER TABLE viewName PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p')
```
In addition, `ALTER TABLE RENAME PARTITION` is unable to handle data source tables, just like the other `ALTER PARTITION` commands. We should issue an exception instead.
### How was this patch tested?
Added a few test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15004 from gatorsmile/altertable.
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).
This PR is to support this new feature.
## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Xin Wu <xinwu@us.ibm.com>
Closes#14842 from xwu0226/SPARK-10747.
### What changes were proposed in this pull request?
As explained in https://github.com/apache/spark/pull/14797:
>Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.
For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.
We should not optimize the query in CTAS more than once. For example,
```Scala
spark.range(99, 101).createOrReplaceTempView("tab1")
val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1"
sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt")
checkAnswer(spark.table("tab2"), sql(sqlStmt))
```
Before this PR, the results do not match
```
== Results ==
!== Correct Answer - 2 == == Spark Answer - 2 ==
![100,100.000000000000000000] [100,null]
[99,99.000000000000000000] [99,99.000000000000000000]
```
After this PR, the results match.
```
+---+----------------------+
|id |num |
+---+----------------------+
|99 |99.000000000000000000 |
|100|100.000000000000000000|
+---+----------------------+
```
In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`.
### How was this patch tested?
Added a test
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15048 from gatorsmile/ctasOptimized.
## What changes were proposed in this pull request?
Point references to spark-packages.org to https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects
This will be accompanied by a parallel change to the spark-website repo, and additional changes to this wiki.
## How was this patch tested?
Jenkins tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#15075 from srowen/SPARK-17445.
## What changes were proposed in this pull request?
CollectLimit.execute() incorrectly omits per-partition limits, leading to performance regressions in case this case is hit (which should not happen in normal operation, but can occur in some cases (see #15068 for one example).
## How was this patch tested?
Regression test in SQLQuerySuite that asserts the number of records scanned from the input RDD.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#15070 from JoshRosen/SPARK-17515.
## What changes were proposed in this pull request?
When there is any Python UDF in the Project between Sort and Limit, it will be collected into TakeOrderedAndProjectExec, ExtractPythonUDFs failed to pull the Python UDFs out because QueryPlan.expressions does not include the expression inside Option[Seq[Expression]].
Ideally, we should fix the `QueryPlan.expressions`, but tried with no luck (it always run into infinite loop). In PR, I changed the TakeOrderedAndProjectExec to no use Option[Seq[Expression]] to workaround it. cc JoshRosen
## How was this patch tested?
Added regression test.
Author: Davies Liu <davies@databricks.com>
Closes#15030 from davies/all_expr.
## What changes were proposed in this pull request?
Check the database warehouse used in Spark UT, and remove the existing database file before run the UT (SPARK-8368).
## How was this patch tested?
Run Spark UT with the command for several times:
./build/sbt -Pyarn -Phadoop-2.6 -Phive -Phive-thriftserver "test-only *HiveSparkSubmitSuit*"
Without the patch, the test case can be passed only at the first time, and always failed from the second time.
With the patch the test case always can be passed correctly.
Author: tone-zhang <tone.zhang@linaro.org>
Closes#14894 from tone-zhang/issue1.
## What changes were proposed in this pull request?
This PR fixes `ColumnVectorUtils.populate` so that Parquet vectorized reader can read partitioned table with dates/timestamps. This works fine with Parquet normal reader.
This is being only called within [VectorizedParquetRecordReader.java#L185](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java#L185).
When partition column types are explicitly given to `DateType` or `TimestampType` (rather than inferring the type of partition column), this fails with the exception below:
```
16/09/01 10:30:07 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 6)
java.lang.ClassCastException: java.lang.Integer cannot be cast to java.sql.Date
at org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate(ColumnVectorUtils.java:89)
at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:185)
at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:204)
at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReader$1.apply(ParquetFileFormat.scala:362)
...
```
## How was this patch tested?
Unit tests in `SQLQuerySuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14919 from HyukjinKwon/SPARK-17354.
## What changes were proposed in this pull request?
In `PreprocessDDL` we will check if table columns are duplicated. However, this checking ignores case sensitivity config(it's always case-sensitive) and lead to different result between `HiveExternalCatalog` and `InMemoryCatalog`. `HiveExternalCatalog` will throw exception because hive metastore is always case-nonsensitive, and `InMemoryCatalog` is fine.
This PR fixes it.
## How was this patch tested?
a new test in DDLSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14994 from cloud-fan/check-dup.
## What changes were proposed in this pull request?
`select size(null)` returns -1 in Hive. In order to be compatible, we should return `-1`.
## How was this patch tested?
unit test in `CollectionFunctionsSuite` and `DataFrameFunctionsSuite`.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#14991 from adrian-wang/size.
## What changes were proposed in this pull request?
When we create a filestream on a directory that has partitioned subdirs (i.e. dir/x=y/), then ListingFileCatalog.allFiles returns the files in the dir as Seq[String] which internally is a Stream[String]. This is because of this [line](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L93), where a LinkedHashSet.values.toSeq returns Stream. Then when the [FileStreamSource](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L79) filters this Stream[String] to remove the seen files, it creates a new Stream[String], which has a filter function that has a $outer reference to the FileStreamSource (in Scala 2.10). Trying to serialize this Stream[String] causes NotSerializableException. This will happened even if there is just one file in the dir.
Its important to note that this behavior is different in Scala 2.11. There is no $outer reference to FileStreamSource, so it does not throw NotSerializableException. However, with a large sequence of files (tested with 10000 files), it throws StackOverflowError. This is because how Stream class is implemented. Its basically like a linked list, and attempting to serialize a long Stream requires *recursively* going through linked list, thus resulting in StackOverflowError.
In short, across both Scala 2.10 and 2.11, serialization fails when both the following conditions are true.
- file stream defined on a partitioned directory
- directory has 10k+ files
The right solution is to convert the seq to an array before writing to the log. This PR implements this fix in two ways.
- Changing all uses for HDFSMetadataLog to ensure Array is used instead of Seq
- Added a `require` in HDFSMetadataLog such that it is never used with type Seq
## How was this patch tested?
Added unit test that test that ensures the file stream source can handle with 10000 files. This tests fails in both Scala 2.10 and 2.11 with different failures as indicated above.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14987 from tdas/SPARK-17372.
## What changes were proposed in this pull request?
In LongToUnsafeRowMap, we use offset of a value as pointer, stored in a array also in the page for chained values. The offset is not portable, because Platform.LONG_ARRAY_OFFSET will be different with different JVM Heap size, then the deserialized LongToUnsafeRowMap will be corrupt.
This PR will change to use portable address (without Platform.LONG_ARRAY_OFFSET).
## How was this patch tested?
Added a test case with random generated keys, to improve the coverage. But this test is not a regression test, that could require a Spark cluster that have at least 32G heap in driver or executor.
Author: Davies Liu <davies@databricks.com>
Closes#14927 from davies/longmap.
## What changes were proposed in this pull request?
This PR adds better error messages for malformed record when reading a JSON file using DataFrameReader.
For example, for query:
```
import org.apache.spark.sql.types._
val corruptRecords = spark.sparkContext.parallelize("""{"a":{, b:3}""" :: Nil)
val schema = StructType(StructField("a", StringType, true) :: Nil)
val jsonDF = spark.read.schema(schema).json(corruptRecords)
```
**Before change:**
We silently replace corrupted line with null
```
scala> jsonDF.show
+----+
| a|
+----+
|null|
+----+
```
**After change:**
Add an explicit warning message:
```
scala> jsonDF.show
16/09/02 14:43:16 WARN JacksonParser: Found at least one malformed records (sample: {"a":{, b:3}). The JSON reader will replace
all malformed records with placeholder null in current PERMISSIVE parser mode.
To find out which corrupted records have been replaced with null, please use the
default inferred schema instead of providing a custom schema.
Code example to print all malformed records (scala):
===================================================
// The corrupted record exists in column _corrupt_record.
val parsedJson = spark.read.json("/path/to/json/file/test.json")
+----+
| a|
+----+
|null|
+----+
```
###
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14929 from clockfly/logwarning_if_schema_not_contain_corrupted_record.
## What changes were proposed in this pull request?
class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression.
```
case class Alias(child: Expression, name: String)(
val exprId: ExprId = NamedExpression.newExprId,
val qualifier: Option[String] = None,
val explicitMetadata: Option[Metadata] = None,
override val isGenerated: java.lang.Boolean = false)
```
The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string.
If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory.
With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356.
## How was this patch tested?
Existing tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14915 from clockfly/json_oom.
## What changes were proposed in this pull request?
Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation.
This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`.
Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups.
## How was this patch tested?
existing tests and new test in `CatalogSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14921 from cloud-fan/check-path.
## What changes were proposed in this pull request?
1. Support generation table-level statistics for
- hive tables in HiveExternalCatalog
- data source tables in HiveExternalCatalog
- data source tables in InMemoryCatalog.
2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side.
3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl.
4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats).
## How was this patch tested?
add unit tests
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>
Closes#14712 from wzhfy/tableStats.
## What changes were proposed in this pull request?
It's really weird that we allow users to specify database in both from table name and to table name
in `ALTER TABLE RENAME TO`, while logically we can't support rename a table to a different database.
Both postgres and MySQL disallow this syntax, it's reasonable to follow them and simply our code.
## How was this patch tested?
new test in `DDLCommandSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14955 from cloud-fan/rename.
## What changes were proposed in this pull request?
Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.
If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.
The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.
## How was this patch tested?
Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.
Author: Srinath Shankar <srinath@databricks.com>
Closes#14866 from srinathshankar/crossjoin.
## What changes were proposed in this pull request?
Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.
For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.
Ideally, we should make all the analyzer rules all idempotent, that may require lots of effort to double checking them one by one (may be not easy).
An easier approach could be never feed a optimized plan into Analyzer, this PR fix the case for RunnableComand, they will be optimized, during execution, the passed `query` will also be passed into QueryExecution again. This PR make these `query` not part of the children, so they will not be optimized and analyzed again.
Right now, we did not know a logical plan is optimized or not, we could introduce a flag for that, and make sure a optimized logical plan will not be analyzed again.
## How was this patch tested?
Added regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#14797 from davies/fix_writer.
## What changes were proposed in this pull request?
This PR is the second step for the following feature:
For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields).
In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs.
In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`.
## How was this patch tested?
Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite`
Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series.
Author: Qifan Pu <qifan.pu@gmail.com>
Author: ooq <qifan.pu@gmail.com>
Closes#14176 from ooq/rowbasedfastaggmap-pr2.
## What changes were proposed in this pull request?
It seems `EqualNullSafe` filter was missed for batch pruneing partitions in cached tables.
It seems supporting this improves the performance roughly 5 times faster.
Running the codes below:
```scala
test("Null-safe equal comparison") {
val N = 20000000
val df = spark.range(N).repartition(20)
val benchmark = new Benchmark("Null-safe equal comparison", N)
df.createOrReplaceTempView("t")
spark.catalog.cacheTable("t")
sql("select id from t where id <=> 1").collect()
benchmark.addCase("Null-safe equal comparison", 10) { _ =>
sql("select id from t where id <=> 1").collect()
}
benchmark.run()
}
```
produces the results below:
**Before:**
```
Running benchmark: Null-safe equal comparison
Running case: Null-safe equal comparison
Stopped after 10 iterations, 2098 ms
Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5
Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz
Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Null-safe equal comparison 204 / 210 98.1 10.2 1.0X
```
**After:**
```
Running benchmark: Null-safe equal comparison
Running case: Null-safe equal comparison
Stopped after 10 iterations, 478 ms
Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5
Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz
Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Null-safe equal comparison 42 / 48 474.1 2.1 1.0X
```
## How was this patch tested?
Unit tests in `PartitionBatchPruningSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14117 from HyukjinKwon/SPARK-16461.
## What changes were proposed in this pull request?
This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`.
If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`.
Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data.
## How was this patch tested?
Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14832 from hvanhovell/SPARK-17263.
## What changes were proposed in this pull request?
This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`.
### Syntax:
```
# Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory.
percentile_approx(col, percentage [, accuracy])
# Returns percentile value array at given percentage value array
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy])
```
### Features:
1. This function supports partial aggregation.
2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint.
3. This function supports window function aggregation.
### Example usages:
```
## Returns the 25th percentile value, with default accuracy
SELECT percentile_approx(col, 0.25) FROM table
## Returns an array of percentile value (25th, 50th, 75th), with default accuracy
SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table
## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error
SELECT percentile_approx(col, 0.25, 100) FROM table
## Returns the 25th, and 50th percentile values, with custom accuracy value 100
SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table
```
### NOTE:
1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)`
2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal.
## How was this patch tested?
Unit test, and Sql query test.
## Acknowledgement
1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14868 from clockfly/appro_percentile_try_2.
## What changes were proposed in this pull request?
according to the discussion in the original PR #10896 and the new approach PR #14876 , we decided to revert these 2 PRs and go with the new approach.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14909 from cloud-fan/revert.
## What changes were proposed in this pull request?
Partial aggregations are generated in `EnsureRequirements`, but the planner fails to
check if partial aggregation satisfies sort requirements.
For the following query:
```
val df2 = (0 to 1000).map(x => (x % 2, x.toString)).toDF("a", "b").createOrReplaceTempView("t2")
spark.sql("select max(b) from t2 group by a").explain(true)
```
Now, the SortAggregator won't insert Sort operator before partial aggregation, this will break sort-based partial aggregation.
```
== Physical Plan ==
SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17])
+- *Sort [a#5 ASC], false, 0
+- Exchange hashpartitioning(a#5, 200)
+- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19])
+- LocalTableScan [a#5, b#6]
```
Actually, a correct plan is:
```
== Physical Plan ==
SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17])
+- *Sort [a#5 ASC], false, 0
+- Exchange hashpartitioning(a#5, 200)
+- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19])
+- *Sort [a#5 ASC], false, 0
+- LocalTableScan [a#5, b#6]
```
## How was this patch tested?
Added tests in `PlannerSuite`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#14865 from maropu/SPARK-17289.
## What changes were proposed in this pull request?
This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions).
It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default).
## How was this patch tested?
Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster).
Author: Davies Liu <davies@databricks.com>
Closes#14607 from davies/repair_batch.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-17271
Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253
`SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects.
eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")`
Expression in required SortOrder:
```
AttributeReference(
name = "col1",
dataType = LongType,
nullable = false
) (exprId = exprId,
qualifier = Some("a")
)
```
Expression in child SortOrder:
```
AttributeReference(
name = "col1",
dataType = LongType,
nullable = false
) (exprId = exprId)
```
Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order.
This PR includes following changes:
- Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals)
- Fixed `EnsureRequirements` to use semantic comparison of SortOrder
## How was this patch tested?
- Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite`
Author: Tejas Patil <tejasp@fb.com>
Closes#14841 from tejasapatil/SPARK-17271_sort_order_equals_bug.
## What changes were proposed in this pull request?
This pr to fix a bug below in sampling with replacement
```
val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b")
df.sample(true, 2.0).withColumn("c", monotonically_increasing_id).select($"c").show
+---+
| c|
+---+
| 0|
| 1|
| 1|
| 1|
| 2|
+---+
```
## How was this patch tested?
Added a test in `DataFrameSuite`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#14800 from maropu/FixSampleBug.
## What changes were proposed in this pull request?
This patch adds a purge interface to MetadataLog, and an implementation in HDFSMetadataLog. The purge function is currently unused, but I will use it to purge old execution and file source logs in follow-up patches. These changes are required in a production structured streaming job that runs for a long period of time.
## How was this patch tested?
Added a unit test case in HDFSMetadataLogSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14802 from petermaxlee/SPARK-17235.
## What changes were proposed in this pull request?
This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values.
## How was this patch tested?
Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14819 from hvanhovell/SPARK-17246.
## What changes were proposed in this pull request?
Before this change, FileStreamSource uses an in-memory hash set to track the list of files processed by the engine. The list can grow indefinitely, leading to OOM or overflow of the hash set.
This patch introduces a new user-defined option called "maxFileAge", default to 24 hours. If a file is older than this age, FileStreamSource will purge it from the in-memory map that was used to track the list of files that have been processed.
## How was this patch tested?
Added unit tests for the underlying utility, and also added an end-to-end test to validate the purge in FileStreamSourceSuite. Also verified the new test cases would fail when the timeout was set to a very large number.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14728 from petermaxlee/SPARK-17165.
### What changes were proposed in this pull request?
Address the comments by yhuai in the original PR: https://github.com/apache/spark/pull/14207
First, issue an exception instead of logging a warning when users specify the partitioning columns without a given schema.
Second, refactor the codes a little.
### How was this patch tested?
Fixed the test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14572 from gatorsmile/followup16552.
## What changes were proposed in this pull request?
This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV.
In more details, this PR,
- Enables the tests for `TimestampType` for JSON and
This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below:
```scala
val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime
println(d.toString)
```
```
Fri Dec 28 00:00:00 KST 899
```
However, since we use `FastDateFormat`, it seems we are safe now.
```scala
val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000")
println(d)
```
```
Tue Jan 01 00:00:00 PST 900
```
- Verifies all unsupported types in CSV
There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types.
## How was this patch tested?
Tests in `JsonHadoopFsRelation` and `CSVSuite`
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14829 from HyukjinKwon/SPARK-16216-followup.
## What changes were proposed in this pull request?
This PR introduces an abstract class `TypedImperativeAggregate` so that an aggregation function of TypedImperativeAggregate can use **arbitrary** user-defined Java object as intermediate aggregation buffer object.
**This has advantages like:**
1. It now can support larger category of aggregation functions. For example, it will be much easier to implement aggregation function `percentile_approx`, which has a complex aggregation buffer definition.
2. It can be used to avoid doing serialization/de-serialization for every call of `update` or `merge` when converting domain specific aggregation object to internal Spark-Sql storage format.
3. It is easier to integrate with other existing monoid libraries like algebird, and supports more aggregation functions with high performance.
Please see `org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMaxAggregate` to find an example of how to defined a `TypedImperativeAggregate` aggregation function.
Please see Java doc of `TypedImperativeAggregate` and Jira ticket SPARK-17187 for more information.
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#14753 from clockfly/object_aggregation_buffer_try_2.
### What changes were proposed in this pull request?
This PR is to fix an incorrect outer join elimination when filter's `isNotNull` constraints is unable to filter out all null-supplying rows. For example, `isnotnull(coalesce(b#227, c#238))`.
Users can hit this error when they try to use `using/natural outer join`, which is converted to a normal outer join with a `coalesce` expression on the `using columns`. For example,
```Scala
val a = Seq((1, 2), (2, 3)).toDF("a", "b")
val b = Seq((2, 5), (3, 4)).toDF("a", "c")
val c = Seq((3, 1)).toDF("a", "d")
val ab = a.join(b, Seq("a"), "fullouter")
ab.join(c, "a").explain(true)
```
The dataframe `ab` is doing `using full-outer join`, which is converted to a normal outer join with a `coalesce` expression. Constraints inference generates a `Filter` with constraints `isnotnull(coalesce(b#227, c#238))`. Then, it triggers a wrong outer join elimination and generates a wrong result.
```
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
:- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
: +- Join FullOuter, (a#226 = a#236)
: :- Project [_1#223 AS a#226, _2#224 AS b#227]
: : +- LocalRelation [_1#223, _2#224]
: +- Project [_1#233 AS a#236, _2#234 AS c#237]
: +- LocalRelation [_1#233, _2#234]
+- Project [_1#243 AS a#246, _2#244 AS d#247]
+- LocalRelation [_1#243, _2#244]
== Optimized Logical Plan ==
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
:- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
: +- Filter isnotnull(coalesce(a#226, a#236))
: +- Join FullOuter, (a#226 = a#236)
: :- LocalRelation [a#226, b#227]
: +- LocalRelation [a#236, c#237]
+- LocalRelation [a#246, d#247]
```
**A note to the `Committer`**, please also give the credit to dongjoon-hyun who submitted another PR for fixing this issue. https://github.com/apache/spark/pull/14580
### How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14661 from gatorsmile/fixOuterJoinElimination.
## What changes were proposed in this pull request?
### Default - ISO 8601
Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below:
- CSV
```
// TimestampType
1414459800000000
// DateType
16673
```
- Json
```
// TimestampType
1970-01-01 11:46:40.0
// DateType
1970-01-01
```
So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed.
So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)).
- For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`)
```
1970-01-01T02:00:01.000-01:00
```
- For `Date` it becomes as below (`yyyy-MM-dd`)
```
1970-01-01
```
### Custom date format option - `dateFormat`
This PR also adds the support to write and read dates and timestamps in a formatted string as below:
- **DateType**
- With `dateFormat` option (e.g. `yyyy/MM/dd`)
```
+----------+
| date|
+----------+
|2015/08/26|
|2014/10/27|
|2016/01/28|
+----------+
```
### Custom date format option - `timestampFormat`
- **TimestampType**
- With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`)
```
+----------------+
| date|
+----------------+
|2015/08/26 18:00|
|2014/10/27 18:30|
|2016/01/28 20:00|
+----------------+
```
## How was this patch tested?
Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14279 from HyukjinKwon/SPARK-16216-json-csv.
## What changes were proposed in this pull request?
This is a sub-task of [SPARK-16283](https://issues.apache.org/jira/browse/SPARK-16283) (Implement percentile_approx SQL function), which moves class QuantileSummaries to project catalyst so that it can be reused when implementing aggregation function `percentile_approx`.
## How was this patch tested?
This PR only does class relocation, class implementation is not changed.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14754 from clockfly/move_QuantileSummaries_to_catalyst.
## What changes were proposed in this pull request?
Fix some typos in comments and test hints
## How was this patch tested?
N/A.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14755 from clockfly/fix_minor_typo.
## What changes were proposed in this pull request?
In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode).
This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType.
It also fix a bug around splitting expression in whole-stage codegen (it should not split them).
## How was this patch tested?
Added benchmark suite.
Author: Davies Liu <davies@databricks.com>
Closes#14692 from davies/split_exprs.
## What changes were proposed in this pull request?
Spark SQL doesn't have its own meta store yet, and use hive's currently. However, hive's meta store has some limitations(e.g. columns can't be too many, not case-preserving, bad decimal type support, etc.), so we have some hacks to successfully store data source table metadata into hive meta store, i.e. put all the information in table properties.
This PR moves these hacks to `HiveExternalCatalog`, tries to isolate hive specific logic in one place.
changes overview:
1. **before this PR**: we need to put metadata(schema, partition columns, etc.) of data source tables to table properties before saving it to external catalog, even the external catalog doesn't use hive metastore(e.g. `InMemoryCatalog`)
**after this PR**: the table properties tricks are only in `HiveExternalCatalog`, the caller side doesn't need to take care of it anymore.
2. **before this PR**: because the table properties tricks are done outside of external catalog, so we also need to revert these tricks when we read the table metadata from external catalog and use it. e.g. in `DescribeTableCommand` we will read schema and partition columns from table properties.
**after this PR**: The table metadata read from external catalog is exactly the same with what we saved to it.
bonus: now we can create data source table using `SessionCatalog`, if schema is specified.
breaks: `schemaStringLengthThreshold` is not configurable anymore. `hive.default.rcfile.serde` is not configurable anymore.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14155 from cloud-fan/catalog-table.
## What changes were proposed in this pull request?
Currently, `NullPropagation` optimizer replaces `COUNT` on null literals in a bottom-up fashion. During that, `WindowExpression` is not covered properly. This PR adds the missing propagation logic.
**Before**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: cast(0 as bigint) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
```
**After**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
+----------------------------------------------------------------------------------------------+
|count((1 + CAST(NULL AS INT))) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)|
+----------------------------------------------------------------------------------------------+
| 0|
+----------------------------------------------------------------------------------------------+
```
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14689 from dongjoon-hyun/SPARK-17098.
## What changes were proposed in this pull request?
This patch fixes a longstanding issue with one of the RelationalGroupedDataset.agg function. Even though the signature accepts vararg of pairs, the underlying implementation turns the seq into a map, and thus not order preserving nor allowing multiple aggregates per column.
This change also allows users to use this function to run multiple different aggregations for a single column, e.g.
```
agg("age" -> "max", "age" -> "count")
```
## How was this patch tested?
Added a test case in DataFrameAggregateSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14697 from petermaxlee/SPARK-17124.
## What changes were proposed in this pull request?
Modifies error message for numeric literals to
Numeric literal <literal> does not fit in range [min, max] for type <T>
## How was this patch tested?
Fixed up the error messages for literals.sql in SqlQueryTestSuite and re-ran via sbt. Also fixed up error messages in ExpressionParserSuite
Author: Srinath Shankar <srinath@databricks.com>
Closes#14721 from srinathshankar/sc4296.
## What changes were proposed in this pull request?
This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including:
- indexing
- array creation
- size
- array_contains
- sort_array
## How was this patch tested?
The patch itself is about adding tests.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14708 from petermaxlee/SPARK-17149.
## What changes were proposed in this pull request?
This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect.
This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite
```
select * from (select * from range(10) limit 5) where id > 3
to
select * from range(10) where id > 3 limit 5
```
## How was this patch tested?
- a unit test case in FilterPushdownSuite
- an end-to-end test in limit.sql
Author: Reynold Xin <rxin@databricks.com>
Closes#14713 from rxin/SPARK-16994.
## What changes were proposed in this pull request?
This patch improves inline table support with the following:
1. Support type coercion.
2. Support using foldable expressions. Previously only literals were supported.
3. Improve error message handling.
4. Improve test coverage.
## How was this patch tested?
Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14676 from petermaxlee/SPARK-16947.
## What changes were proposed in this pull request?
This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception:
```
org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null).
```
The problem is that division type coercion did not take null type into account.
## How was this patch tested?
A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14695 from petermaxlee/SPARK-17117.
## What changes were proposed in this pull request?
This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`.
## How was this patch tested?
Unit tests.
cc hvanhovell
Author: Eric Liang <ekl@databricks.com>
Closes#14656 from ericl/sc-4309.
## What changes were proposed in this pull request?
This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation.
Note that the pull request was initially done by viirya.
## How was this patch tested?
Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator.
Author: Reynold Xin <rxin@databricks.com>
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14576 from rxin/reduceAggregator.
## What changes were proposed in this pull request?
A TreeNodeException is thrown when executing the following minimal example in Spark 2.0.
import spark.implicits._
case class test (x: Int, q: Int)
val d = Seq(1).toDF("x")
d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error.
We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14648 from viirya/flat-mapping.
## What changes were proposed in this pull request?
Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception. For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have
```
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211)
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```
This is basically useless, as it is location where the StreamingQueryException was defined. What we want is
Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated
- The actual exception could either be a SparkException, or an arbitrary exception.
- SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant.
- For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant.
- When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra
- Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object.
With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero
at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153)
at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226)
at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
...
```
It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace.
```
java.lang.RuntimeException: this is the exception message
at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103)
at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316)
at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
at scala.collection.Iterator$class.foreach(Iterator.scala:893)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25)
at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313)
at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197)
at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)
at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187)
at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```
Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental.
## How was this patch tested?
Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14675 from tdas/SPARK-17096.
## What changes were proposed in this pull request?
We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default).
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14679 from cloud-fan/json.
## What changes were proposed in this pull request?
The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code.
This PR unifies these access paths.
## How was this patch tested?
(Existing tests)
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14685 from hvanhovell/SPARK-17106.
## What changes were proposed in this pull request?
This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method.
Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](fa244e5a90).
## How was this patch tested?
Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#14670 from kiszk/SPARK-15285-2.
## What changes were proposed in this pull request?
This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals.
Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved.
### Before this change
Ordinal is stored as `Literal` expression
```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [1 ASC], true
+- 'Aggregate [1], ['a]
+- 'UnresolvedRelation `t
```
For query:
```
scala> Seq(1).toDF("a").createOrReplaceTempView("t")
scala> sql("select count(a), a from t group by 2 having a > 0").show
```
During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is:
```
'Filter ('a > 0)
+- Aggregate [2], [count(1) AS count(1)#83L, a#81]
+- LocalRelation [value#7 AS a#9]
```
Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved!
### After this change
Ordinals are stored as `UnresolvedOrdinal`.
```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [unresolvedordinal(1) ASC], true
+- 'Aggregate [unresolvedordinal(1)], ['a]
+- 'UnresolvedRelation `t`
```
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14616 from clockfly/spark-16955.
## What changes were proposed in this pull request?
`CatalogStorageFormat.properties` can be used in 2 ways:
1. for hive tables, it stores the serde properties.
2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc.
however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14506 from cloud-fan/table-prop.
Both core and sql have slightly different code that does variable substitution
of config values. This change refactors that code and encapsulates the logic
of reading config values and expading variables in a new helper class, which
can be configured so that both core and sql can use it without losing existing
functionality, and allows for easier testing and makes it easier to add more
features in the future.
Tested with existing and new unit tests, and by running spark-shell with
some configs referencing variables and making sure it behaved as expected.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#14468 from vanzin/SPARK-16671.
## What changes were proposed in this pull request?
Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied.
See also https://github.com/apache/spark/pull/14602
This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556
## How was this patch tested?
Jenkins test, with new case reproducing the bug
Author: Sean Owen <sowen@cloudera.com>
Closes#14630 from srowen/SPARK-16966.2.
## What changes were proposed in this pull request?
In the PR, we just allow the user to add additional options when create a new table in JDBC writer.
The options can be table_options or partition_options.
E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8"
Here is the usage example:
```
df.write.option("createTableOptions", "ENGINE=InnoDB DEFAULT CHARSET=utf8").jdbc(...)
```
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
will apply test result soon.
Author: GraceH <93113783@qq.com>
Closes#14559 from GraceH/jdbc_options.
## What changes were proposed in this pull request?
This PR changes the CTE resolving rule to use only **forward-declared** tables in order to prevent infinite loops. More specifically, new logic is like the following.
* Resolve CTEs in `WITH` clauses first before replacing the main SQL body.
* When resolving CTEs, only forward-declared CTEs or base tables are referenced.
- Self-referencing is not allowed any more.
- Cross-referencing is not allowed any more.
**Reported Error Scenarios**
```scala
scala> sql("WITH t AS (SELECT 1 FROM t) SELECT * FROM t")
java.lang.StackOverflowError
...
scala> sql("WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1, t2")
java.lang.StackOverflowError
...
```
Note that `t`, `t1`, and `t2` are not declared in database. Spark falls into infinite loops before resolving table names.
## How was this patch tested?
Pass the Jenkins tests with new two testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14397 from dongjoon-hyun/SPARK-16771-TREENODE.
## What changes were proposed in this pull request?
This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals.
This allows the parser to parse the minimal value for each data type, e.g. "-32768S".
## How was this patch tested?
Updated test cases.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14608 from petermaxlee/SPARK-17013.
## What changes were proposed in this pull request?
Currently, Spark ignores path names starting with underscore `_` and `.`. This causes read-failures for the column-partitioned file data sources whose partition column names starts from '_', e.g. `_col`.
**Before**
```scala
scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
scala> spark.read.parquet("/tmp/parquet")
org.apache.spark.sql.AnalysisException: Unable to infer schema for ParquetFormat at /tmp/parquet20. It must be specified manually;
```
**After**
```scala
scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
scala> spark.read.parquet("/tmp/parquet")
res2: org.apache.spark.sql.DataFrame = [id: bigint, _locality_code: int]
```
## How was this patch tested?
Pass the Jenkins with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14585 from dongjoon-hyun/SPARK-16975-PARQUET.
## What changes were proposed in this pull request?
This patch adds literals.sql for testing literal parsing end-to-end in SQL.
## How was this patch tested?
The patch itself is only about adding test cases.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14598 from petermaxlee/SPARK-17018-2.
## What changes were proposed in this pull request?
This patch adds three test files:
1. arithmetic.sql.out
2. order-by-ordinal.sql
3. group-by-ordinal.sql
This includes https://github.com/apache/spark/pull/14594.
## How was this patch tested?
This is a test case change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14595 from petermaxlee/SPARK-17015.
## What changes were proposed in this pull request?
This patch adds exception testing to SQLQueryTestSuite. When there is an exception in query execution, the query result contains the the exception class along with the exception message.
As part of this, I moved some additional test cases for limit from SQLQuerySuite over to SQLQueryTestSuite.
## How was this patch tested?
This is a test harness change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14592 from petermaxlee/SPARK-17011.
## What changes were proposed in this pull request?
This patch moves all the test data files in sql/core/src/test/resources to sql/core/src/test/resources/test-data, so we don't clutter the top level sql/core/src/test/resources. Also deleted sql/core/src/test/resources/old-repeated.parquet since it is no longer used.
The change will make it easier to spot sql-tests directory.
## How was this patch tested?
This is a test-only change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14589 from petermaxlee/SPARK-17007.
## What changes were proposed in this pull request?
This patch enhances SQLQueryTestSuite in two ways:
1. SPARK-17009: Use a new SparkSession for each test case to provide stronger isolation (e.g. config changes in one test case does not impact another). That said, we do not currently isolate catalog changes.
2. SPARK-17008: Normalize query output using sorting, inspired by HiveComparisonTest.
I also ported a few new test cases over from SQLQuerySuite.
## How was this patch tested?
This is a test harness update.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14590 from petermaxlee/SPARK-17008.
## What changes were proposed in this pull request?
The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader.
The benchmark that excludes the time of writing Parquet file:
test("Benchmark for Parquet") {
val N = 500 << 12
withParquetTable((0 until N).map(i => (101, i)), "t") {
val benchmark = new Benchmark("Parquet reader", N)
benchmark.addCase("reading Parquet file", 10) { iter =>
sql("SELECT _1 FROM t where t._1 < 100").collect()
}
benchmark.run()
}
}
`withParquetTable` in default will run tests for vectorized reader non-vectorized readers. I only let it run vectorized reader.
When we set the block size of parquet as 1024 to have multiple row groups. The benchmark is:
Before this patch:
The retrieved row groups: 8063
Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
Intel(R) Core(TM) i7-5557U CPU 3.10GHz
Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
reading Parquet file 825 / 1233 2.5 402.6 1.0X
After this patch:
The retrieved row groups: 0
Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
Intel(R) Core(TM) i7-5557U CPU 3.10GHz
Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
reading Parquet file 306 / 503 6.7 149.6 1.0X
Next, I run the benchmark for non-pushdown case using the same benchmark code but with disabled pushdown configuration. This time the parquet block size is default value.
Before this patch:
Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
Intel(R) Core(TM) i7-5557U CPU 3.10GHz
Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
reading Parquet file 136 / 238 15.0 66.5 1.0X
After this patch:
Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
Intel(R) Core(TM) i7-5557U CPU 3.10GHz
Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
reading Parquet file 124 / 193 16.5 60.7 1.0X
For non-pushdown case, from the results, I think this patch doesn't affect normal code path.
I've manually output the `totalRowCount` in `SpecificParquetRecordReaderBase` to see if this patch actually filter the row-groups. When running the above benchmark:
After this patch:
`totalRowCount = 0`
Before this patch:
`totalRowCount = 1024000`
## How was this patch tested?
Existing tests should be passed.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13701 from viirya/vectorized-reader-push-down-filter2.
## What changes were proposed in this pull request?
Fix the construction of the file path. Previous way of construction caused the creation of incorrect path on Windows.
## How was this patch tested?
Run SQL unit tests on Windows
Author: avulanov <nashb@yandex.ru>
Closes#13868 from avulanov/SPARK-15899-file.
## What changes were proposed in this pull request?
This patch introduces SQLQueryTestSuite, a basic framework for end-to-end SQL test cases defined in spark/sql/core/src/test/resources/sql-tests. This is a more standard way to test SQL queries end-to-end in different open source database systems, because it is more manageable to work with files.
This is inspired by HiveCompatibilitySuite, but simplified for general Spark SQL tests. Once this is merged, I can work towards porting SQLQuerySuite over, and eventually also move the existing HiveCompatibilitySuite to use this framework.
Unlike HiveCompatibilitySuite, SQLQueryTestSuite compares both the output schema and the output data (in string form).
When there is a mismatch, the error message looks like the following:
```
[info] - blacklist.sql !!! IGNORED !!!
[info] - number-format.sql *** FAILED *** (2 seconds, 405 milliseconds)
[info] Expected "...147483648 -214748364[8]", but got "...147483648 -214748364[9]" Result should match for query #1 (SQLQueryTestSuite.scala:171)
[info] org.scalatest.exceptions.TestFailedException:
[info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495)
[info] at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
[info] at org.scalatest.Assertions$class.assertResult(Assertions.scala:1171)
```
## How was this patch tested?
This is a test infrastructure change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14472 from petermaxlee/SPARK-16866.
## What changes were proposed in this pull request?
Doc that regexp_extract returns empty string when regex or group does not match
## How was this patch tested?
Jenkins test, with a few new test cases
Author: Sean Owen <sowen@cloudera.com>
Closes#14525 from srowen/SPARK-16324.
#### What changes were proposed in this pull request?
When we do not turn on the Hive Support, the following query generates a confusing error message by Planner:
```Scala
sql("CREATE TABLE t2 SELECT a, b from t1")
```
```
assertion failed: No plan for CreateTable CatalogTable(
Table: `t2`
Created: Tue Aug 09 23:45:32 PDT 2016
Last Access: Wed Dec 31 15:59:59 PST 1969
Type: MANAGED
Provider: hive
Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists
+- Relation[a#19L,b#20L] parquet
java.lang.AssertionError: assertion failed: No plan for CreateTable CatalogTable(
Table: `t2`
Created: Tue Aug 09 23:45:32 PDT 2016
Last Access: Wed Dec 31 15:59:59 PST 1969
Type: MANAGED
Provider: hive
Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists
+- Relation[a#19L,b#20L] parquet
```
This PR is to issue a better error message:
```
Hive support is required to use CREATE Hive TABLE AS SELECT
```
#### How was this patch tested?
Added test cases in `DDLSuite.scala`
Author: gatorsmile <gatorsmile@gmail.com>
Closes#13886 from gatorsmile/createCatalogedTableAsSelect.
## What changes were proposed in this pull request?
This PR adds `MINUS` set operator which is equivalent `EXCEPT DISTINCT`. This will slightly improve the compatibility with Oracle.
## How was this patch tested?
Pass the Jenkins with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14570 from dongjoon-hyun/SPARK-10601.
## What changes were proposed in this pull request?
MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system.
Another syntax is: ALTER TABLE table RECOVER PARTITIONS
The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed).
## How was this patch tested?
Added unit tests for it and Hive compatibility test suite.
Author: Davies Liu <davies@databricks.com>
Closes#14500 from davies/repair_table.
## What changes were proposed in this pull request?
This PR fixes the following to make `checkAnswer` raise `TestFailedException` again instead of `java.util.NoSuchElementException: key not found: TZ` in the environments without `TZ` variable. Also, this PR adds `QueryTestSuite` class for testing `QueryTest` itself.
```scala
- |Timezone Env: ${sys.env("TZ")}
+ |Timezone Env: ${sys.env.getOrElse("TZ", "")}
```
## How was this patch tested?
Pass the Jenkins tests with a new test suite.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14528 from dongjoon-hyun/SPARK-16940.
## What changes were proposed in this pull request?
Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.
## How was this patch tested?
Existing tests.
Author: Holden Karau <holden@us.ibm.com>
Closes#14407 from holdenk/SPARK-16779.
### What changes were proposed in this pull request?
Currently, the `refreshTable` API is always case sensitive.
When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like
```
Job aborted due to stage failure:
Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost):
java.io.FileNotFoundException:
File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist
```
This PR is to fix the issue.
### How was this patch tested?
Added a test case.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14523 from gatorsmile/refreshTempTable.
## What changes were proposed in this pull request?
This patch fixes the incorrect results in the rule ResolveSubquery in Catalyst's Analysis phase by returning an error message when the LIMIT is found in the path from the parent table to the correlated predicate in the subquery.
## How was this patch tested?
./dev/run-tests
a new unit test on the problematic pattern.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#14411 from nsyca/master.
## What changes were proposed in this pull request?
This PR aims to fix a build error on branch 1.6 at 8d87252087, but I think we had better have this consistently in master branch, too. It's because there exist other ongoing PR (https://github.com/apache/spark/pull/14525) about this.
https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-1.6-compile-maven-with-yarn-2.3/286/console
```scala
[error] /home/jenkins/workspace/spark-branch-1.6-compile-maven-with-yarn-2.3/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala:82: value toDF is not a member of Seq[String]
[error] val df = Seq("aaaac").toDF("s")
[error] ^
```
## How was this patch tested?
After passing Jenkins, run compilation test on branch 1.6.
```
build/mvn -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
```
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14526 from dongjoon-hyun/SPARK-16939.
## What changes were proposed in this pull request?
regexp_extract actually returns null when it shouldn't when a regex matches but the requested optional group did not. This makes it return an empty string, as apparently designed.
## How was this patch tested?
Additional unit test
Author: Sean Owen <sowen@cloudera.com>
Closes#14504 from srowen/SPARK-16409.
## What changes were proposed in this pull request?
The java.net.URL class has a globally synchronized Hashtable, which limits the throughput of any single executor doing lots of calls to parse_url(). Tests have shown that a 36-core machine can only get to 10% CPU use because the threads are locked most of the time.
This patch switches to java.net.URI which has less features than java.net.URL but focuses on URI parsing, which is enough for parse_url().
New tests were added to make sure a few common edge cases didn't change behaviour.
https://issues.apache.org/jira/browse/SPARK-16826
## How was this patch tested?
I've kept the old URL code commented for now, so that people can verify that the new unit tests do pass with java.net.URL.
Thanks to srowen for the help!
Author: Sylvain Zimmer <sylvain@sylvainzimmer.com>
Closes#14488 from sylvinus/master.
## What changes were proposed in this pull request?
Spark will convert **BooleanType** to **BIT(1)**, **LongType** to **BIGINT**, **ByteType** to **BYTE** when saving DataFrame to Oracle, but Oracle does not support BIT, BIGINT and BYTE types.
This PR is convert following _Spark Types_ to _Oracle types_ refer to [Oracle Developer's Guide](https://docs.oracle.com/cd/E19501-01/819-3659/gcmaz/)
Spark Type | Oracle
----|----
BooleanType | NUMBER(1)
IntegerType | NUMBER(10)
LongType | NUMBER(19)
FloatType | NUMBER(19, 4)
DoubleType | NUMBER(19, 4)
ByteType | NUMBER(3)
ShortType | NUMBER(5)
## How was this patch tested?
Add new tests in [JDBCSuite.scala](22b0c2a422 (diff-dc4b58851b084b274df6fe6b189db84d)) and [OracleDialect.scala](22b0c2a422 (diff-5e0cadf526662f9281aa26315b3750ad))
Author: Yuming Wang <wgyumg@gmail.com>
Closes#14377 from wangyum/SPARK-16625.
## What changes were proposed in this pull request?
we have various logical plans for CREATE TABLE and CTAS: `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateHiveTableAsSelectLogicalPlan`. This PR unifies them to reduce the complexity and centralize the error handling.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14482 from cloud-fan/table.
## What changes were proposed in this pull request?
DatasetBenchmark compares the performances of RDD, DataFrame and Dataset while running the same operations. However, there are two problems that make the comparisons unfair.
1) In backToBackMap test case, only DataFrame implementation executes less work compared to RDD or Dataset implementations. This test case processes Long+String pairs, but the output from the DataFrame implementation does not include String part while RDD or Dataset generates Long+String pairs as output. This difference significantly changes the performance characteristics due to the String manipulation and creation overheads.
2) In back-to-back map and back-to-back filter test cases, `map` or `filter` operation is executed only once regardless of `numChains` parameter for RDD. Hence the execution times for RDD have been largely underestimated.
Of course, these issues do not affect Spark users, but it may confuse Spark developers.
## How was this patch tested?
By executing the DatasetBenchmark
Author: Hiroshi Inoue <inouehrs@jp.ibm.com>
Closes#13459 from inouehrs/fix_benchmark_fairness.
## What changes were proposed in this pull request?
This patch fix the overflow in LongToUnsafeRowMap when the range of key is very wide (the key is much much smaller then minKey, for example, key is Long.MinValue, minKey is > 0).
## How was this patch tested?
Added regression test (also for SPARK-16740)
Author: Davies Liu <davies@databricks.com>
Closes#14464 from davies/fix_overflow.
## What changes were proposed in this pull request?
For DataSet typed select:
```
def select[U1: Encoder](c1: TypedColumn[T, U1]): Dataset[U1]
```
If type T is a case class or a tuple class that is not atomic, the resulting logical plan's schema will mismatch with `Dataset[T]` encoder's schema, which will cause encoder error and throw AnalysisException.
### Before change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A])
org.apache.spark.sql.AnalysisException: cannot resolve '`a`' given input columns: [_2];
..
```
### After change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A]).show
+---+---+
| a| b|
+---+---+
| 1| 2|
+---+---+
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14474 from clockfly/SPARK-16853.
## What changes were proposed in this pull request?
Replace deprecated ParquetWriter with the new builders
## How was this patch tested?
Existing tests
Author: Holden Karau <holden@us.ibm.com>
Closes#14419 from holdenk/SPARK-16814-fix-deprecated-parquet-constructor-usage.
## What changes were proposed in this pull request?
Partition discovery is rather expensive, so we should do it at execution time instead of during physical planning. Right now there is not much benefit since ListingFileCatalog will read scan for all partitions at planning time anyways, but this can be optimized in the future. Also, there might be more information for partition pruning not available at planning time.
This PR moves a lot of the file scan logic from planning to execution time. All file scan operations are handled by `FileSourceScanExec`, which handles both batched and non-batched file scans. This requires some duplication with `RowDataSourceScanExec`, but is probably worth it so that `FileSourceScanExec` does not need to depend on an input RDD.
TODO: In another pr, move DataSourceScanExec to it's own file.
## How was this patch tested?
Existing tests (it might be worth adding a test that catalog.listFiles() is delayed until execution, but this can be delayed until there is an actual benefit to doing so).
Author: Eric Liang <ekl@databricks.com>
Closes#14241 from ericl/refactor.
## What changes were proposed in this pull request?
In Spark 1.6 (with Hive support) we could use `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions as literals (without adding braces), for example:
```SQL
select /* Spark 1.6: */ current_date, /* Spark 1.6 & Spark 2.0: */ current_date()
```
This was accidentally dropped in Spark 2.0. This PR reinstates this functionality.
## How was this patch tested?
Added a case to ExpressionParserSuite.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14442 from hvanhovell/SPARK-16836.
## What changes were proposed in this pull request?
Change to non-deprecated constructor for SQLContext.
## How was this patch tested?
Existing tests
Author: Holden Karau <holden@us.ibm.com>
Closes#14406 from holdenk/SPARK-16778-fix-use-of-deprecated-SQLContext-constructor.
## What changes were proposed in this pull request?
It is useful to log the timezone when query result does not match, especially on build machines that have different timezone from AMPLab Jenkins.
## How was this patch tested?
This is a test-only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#14413 from rxin/SPARK-16805.
## What changes were proposed in this pull request?
`StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14363 from cloud-fan/column.
## What changes were proposed in this pull request?
This fixes a bug wherethe file scan operator does not take into account partition pruning in its implementation of `sameResult()`. As a result, executions may be incorrect on self-joins over the same base file relation.
The patch here is minimal, but we should reconsider relying on `metadata` for implementing sameResult() in the future, as string representations may not be uniquely identifying.
cc rxin
## How was this patch tested?
Unit tests.
Author: Eric Liang <ekl@databricks.com>
Closes#14425 from ericl/spark-16818.
## What changes were proposed in this pull request?
Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation
## How was this patch tested?
Jenkins
Author: Sean Owen <sowen@cloudera.com>
Closes#14332 from srowen/SPARK-16694.
## What changes were proposed in this pull request?
We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException.
## How was this patch tested?
New unit test
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14395 from tdas/SPARK-16748.
## What changes were proposed in this pull request?
f12f11e578 introduced this bug, missed foreach as map
## How was this patch tested?
Test added
Author: Wesley Tang <tangmingjun@mininglamp.com>
Closes#14324 from breakdawn/master.
## What changes were proposed in this pull request?
The query with having condition that contains grouping by column will be failed during analysis. E.g.,
create table tbl(a int, b string);
select count(b) from tbl group by a + 1 having a + 1 = 2;
Having condition should be able to use grouping by column.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14296 from viirya/having-contains-grouping-column.
#### What changes were proposed in this pull request?
Currently, in Spark SQL, the initial creation of schema can be classified into two groups. It is applicable to both Hive tables and Data Source tables:
**Group A. Users specify the schema.**
_Case 1 CREATE TABLE AS SELECT_: the schema is determined by the result schema of the SELECT clause. For example,
```SQL
CREATE TABLE tab STORED AS TEXTFILE
AS SELECT * from input
```
_Case 2 CREATE TABLE_: users explicitly specify the schema. For example,
```SQL
CREATE TABLE jsonTable (_1 string, _2 string)
USING org.apache.spark.sql.json
```
**Group B. Spark SQL infers the schema at runtime.**
_Case 3 CREATE TABLE_. Users do not specify the schema but the path to the file location. For example,
```SQL
CREATE TABLE jsonTable
USING org.apache.spark.sql.json
OPTIONS (path '${tempDir.getCanonicalPath}')
```
Before this PR, Spark SQL does not store the inferred schema in the external catalog for the cases in Group B. When users refreshing the metadata cache, accessing the table at the first time after (re-)starting Spark, Spark SQL will infer the schema and store the info in the metadata cache for improving the performance of subsequent metadata requests. However, the runtime schema inference could cause undesirable schema changes after each reboot of Spark.
This PR is to store the inferred schema in the external catalog when creating the table. When users intend to refresh the schema after possible changes on external files (table location), they issue `REFRESH TABLE`. Spark SQL will infer the schema again based on the previously specified table location and update/refresh the schema in the external catalog and metadata cache.
In this PR, we do not use the inferred schema to replace the user specified schema for avoiding external behavior changes . Based on the design, user-specified schemas (as described in Group A) can be changed by ALTER TABLE commands, although we do not support them now.
#### How was this patch tested?
TODO: add more cases to cover the changes.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14207 from gatorsmile/userSpecifiedSchema.
## What changes were proposed in this pull request?
Spark 1.x supports using the Hive type name as function names for doing casts, e.g.
```sql
SELECT int(1.0);
SELECT string(2.0);
```
The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed.
This patch implements function aliases using an analyzer rule for the following cast functions:
- boolean
- tinyint
- smallint
- int
- bigint
- float
- double
- decimal
- date
- timestamp
- binary
- string
## How was this patch tested?
Added end-to-end tests in SQLCompatibilityFunctionSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14364 from petermaxlee/SPARK-16730-2.
## What changes were proposed in this pull request?
Currently there are 2 inconsistence:
1. for data source table, we only print partition names, for hive table, we also print partition schema. After this PR, we will always print schema
2. if column doesn't have comment, data source table will print empty string, hive table will print null. After this PR, we will always print null
## How was this patch tested?
new test in `HiveDDLSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14302 from cloud-fan/minor3.
## What changes were proposed in this pull request?
finish the TODO, create a new expression `ExternalMapToCatalyst` to iterate the map directly.
## How was this patch tested?
new test in `JavaDatasetSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14344 from cloud-fan/java-map.
## What changes were proposed in this pull request?
We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect.
Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14327 from viirya/fix-sample-pushdown.
## What changes were proposed in this pull request?
This PR contains three changes.
First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below:
1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value.
2. If the offset row does not exist, the default value will be used.
3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change).
Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist.
Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved.
## How was this patch tested?
New tests in SQLWindowFunctionSuite
Author: Yin Huai <yhuai@databricks.com>
Closes#14284 from yhuai/lead-lag.
## What changes were proposed in this pull request?
Current fix for deadlock disables interrupts in the StreamExecution which getting offsets for all sources, and when writing to any metadata log, to avoid potential deadlocks in HDFSMetadataLog(see JIRA for more details). However, disabling interrupts can have unintended consequences in other sources. So I am making the fix more narrow, by disabling interrupt it only in the HDFSMetadataLog. This is a narrower fix for something risky like disabling interrupt.
## How was this patch tested?
Existing tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14292 from tdas/SPARK-14131.
## What changes were proposed in this pull request?
It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698.
Field name having dots throws an exception. For example the codes below:
```scala
val path = "/tmp/path"
val json =""" {"a.b":"data"}"""
spark.sparkContext
.parallelize(json :: Nil)
.saveAsTextFile(path)
spark.read.json(path).collect()
```
throws an exception as below:
```
Unable to resolve a.b given [a.b];
org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b];
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
at scala.Option.getOrElse(Option.scala:121)
```
This problem was introduced in 17eec0a71b (diff-27c76f96a7b2733ecfd6f46a1716e153R121)
When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields.
For example, this throws an exception. (**Loading JSON from RDD is fine**)
```scala
val json =""" {"a.b":"data"}"""
val rdd = spark.sparkContext.parallelize(json :: Nil)
spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true))))
.json(rdd).select("`a.b`").printSchema()
```
as below:
```
cannot resolve '```a.b```' given input columns: [`a.b`];
org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`];
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
```
## How was this patch tested?
Unit tests in `FileSourceStrategySuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14339 from HyukjinKwon/SPARK-16698-regression.
## What changes were proposed in this pull request?
This patch adds an explicit test for [SPARK-14217] by setting the parquet dictionary and page size the generated parquet file spans across 3 pages (within a single row group) where the first page is dictionary encoded and the remaining two are plain encoded.
## How was this patch tested?
1. ParquetEncodingSuite
2. Also manually tested that this test fails without https://github.com/apache/spark/pull/12279
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#14304 from sameeragarwal/hybrid-encoding-test.
## What changes were proposed in this pull request?
It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14331 from cloud-fan/check.
## What changes were proposed in this pull request?
we also store data source table options in this field, it's unreasonable to call it `serdeProperties`.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14283 from cloud-fan/minor1.
## What changes were proposed in this pull request?
This PR adds a boolean option, `truncate`, for `SaveMode.Overwrite` of JDBC DataFrameWriter. If this option is `true`, it try to take advantage of `TRUNCATE TABLE` instead of `DROP TABLE`. This is a trivial option, but will provide great **convenience** for BI tool users based on RDBMS tables generated by Spark.
**Goal**
- Without `CREATE/DROP` privilege, we can save dataframe to database. Sometime these are not allowed for security.
- It will preserve the existing table information, so users can add and keep some additional `INDEX` and `CONSTRAINT`s for the table.
- Sometime, `TRUNCATE` is faster than the combination of `DROP/CREATE`.
**Supported DBMS**
The following is `truncate`-option support table. Due to the different behavior of `TRUNCATE TABLE` among DBMSs, it's not always safe to use `TRUNCATE TABLE`. Spark will ignore the `truncate` option for **unknown** and **some** DBMS with **default CASCADING** behavior. Newly added JDBCDialect should implement corresponding function to support `truncate` option additionally.
Spark Dialects | `truncate` OPTION SUPPORT
---------------|-------------------------------
MySQLDialect | O
PostgresDialect | X
DB2Dialect | O
MsSqlServerDialect | O
DerbyDialect | O
OracleDialect | O
**Before (TABLE with INDEX case)**: SparkShell & MySQL CLI are interleaved intentionally.
```scala
scala> val (url, prop)=("jdbc:mysql://localhost:3306/temp?useSSL=false", new java.util.Properties)
scala> prop.setProperty("user","root")
scala> df.write.mode("overwrite").jdbc(url, "table_with_index", prop)
scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id | bigint(20) | NO | | NULL | |
+-------+------------+------+-----+---------+-------+
mysql> CREATE UNIQUE INDEX idx_id ON table_with_index(id);
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id | bigint(20) | NO | PRI | NULL | |
+-------+------------+------+-----+---------+-------+
scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id | bigint(20) | NO | | NULL | |
+-------+------------+------+-----+---------+-------+
```
**After (TABLE with INDEX case)**
```scala
scala> spark.range(10).write.mode("overwrite").option("truncate", true).jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id | bigint(20) | NO | PRI | NULL | |
+-------+------------+------+-----+---------+-------+
```
**Error Handling**
- In case of exceptions, Spark will not retry. Users should turn off the `truncate` option.
- In case of schema change:
- If one of the column names changes, this will raise exceptions intuitively.
- If there exists only type difference, this will work like Append mode.
## How was this patch tested?
Pass the Jenkins tests with a updated testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14086 from dongjoon-hyun/SPARK-16410.
## What changes were proposed in this pull request?
after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14318 from cloud-fan/minor4.
### What changes were proposed in this pull request?
**Issue 1: Silent Ignorance of Bucket Specification When Creating Table Using Schema Inference**
When creating a data source table without explicit specification of schema or SELECT clause, we silently ignore the bucket specification (CLUSTERED BY... SORTED BY...) in [the code](ce3b98bae2/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala (L339-L354)).
For example,
```SQL
CREATE TABLE jsonTable
USING org.apache.spark.sql.json
OPTIONS (
path '${tempDir.getCanonicalPath}'
)
CLUSTERED BY (inexistentColumnA) SORTED BY (inexistentColumnB) INTO 2 BUCKETS
```
This PR captures it and issues an error message.
**Issue 2: Got a run-time `java.lang.ArithmeticException` when num of buckets is set to zero.**
For example,
```SQL
CREATE TABLE t USING PARQUET
OPTIONS (PATH '${path.toString}')
CLUSTERED BY (a) SORTED BY (b) INTO 0 BUCKETS
AS SELECT 1 AS a, 2 AS b
```
The exception we got is
```
ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1.0 (TID 2)
java.lang.ArithmeticException: / by zero
```
This PR captures the misuse and issues an appropriate error message.
### How was this patch tested?
Added a test case in DDLSuite
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14210 from gatorsmile/createTableWithoutSchema.