## 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.