Commit graph

3416 commits

Author SHA1 Message Date
Liang-Chi Hsieh 0f2c0b53e8 [SPARK-26837][SQL] Pruning nested fields from object serializers
## What changes were proposed in this pull request?

In SPARK-26619, we make change to prune unnecessary individual serializers when serializing objects. This is extension to SPARK-26619. We can further prune nested fields from object serializers if they are not used.

For example, in following query, we only use one field in a struct column:

```scala
val data = Seq((("a", 1), 1), (("b", 2), 2), (("c", 3), 3))
val df = data.toDS().map(t => (t._1, t._2 + 1)).select("_1._1")
```

So, instead of having a serializer to create a two fields struct, we can prune unnecessary field from it. This is what this PR proposes to do.

In order to make this change conservative and safer, a SQL config is added to control it. It is disabled by default.

TODO: Support to prune nested fields inside MapType's key and value.

## How was this patch tested?

Added tests.

Closes #23740 from viirya/nested-pruning-serializer-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 12:45:24 +08:00
Maxim Gekk a2a41b7bf2 [SPARK-26978][CORE][SQL] Avoid magic time constants
## What changes were proposed in this pull request?

In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.

## How was this patch tested?

The changes are tested by existing test suites.

Closes #23878 from MaxGekk/magic-time-constants.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-26 09:08:12 -06:00
Maxim Gekk 75c48ac36d [SPARK-26908][SQL] Fix DateTimeUtils.toMillis and millisToDays
## What changes were proposed in this pull request?

The `DateTimeUtils.toMillis` can produce inaccurate result for some negative values (timestamps before epoch). The error can be around 1ms. In the PR, I propose to use `Math.floorDiv` in casting microseconds to milliseconds, and milliseconds to days since epoch.

## How was this patch tested?

Added new test to `DateTimeUtilsSuite`, and tested by `CastSuite` as well.

Closes #23815 from MaxGekk/micros-to-millis.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-23 11:35:11 -06:00
Maxim Gekk d0f2fd05e1 [SPARK-26903][SQL] Remove the TimeZone cache
## What changes were proposed in this pull request?

In the PR, I propose to convert time zone string to `TimeZone` by converting it to `ZoneId` which uses `ZoneOffset` internally. The `ZoneOffset` class of JDK 8 has a cache already: http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/687fd7c7986d/src/share/classes/java/time/ZoneOffset.java#l205 . In this way, there is no need to support cache of time zones in Spark.

The PR removes `computedTimeZones` from `DateTimeUtils`, and uses `ZoneId.of` to convert time zone id string to `ZoneId` and to `TimeZone` at the end.

## How was this patch tested?

The changes were tested by

Closes #23812 from MaxGekk/timezone-cache.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-23 09:44:22 -06:00
Takeshi Yamamuro 967e4cb011 [SPARK-26215][SQL] Define reserved/non-reserved keywords based on the ANSI SQL standard
## What changes were proposed in this pull request?
This pr targeted to define reserved/non-reserved keywords for Spark SQL based on the ANSI SQL standards and the other database-like systems (e.g., PostgreSQL). We assume that they basically follow the ANSI SQL-2011 standard, but it is slightly different between each other. Therefore, this pr documented all the keywords in `docs/sql-reserved-and-non-reserved-key-words.md`.

NOTE: This pr only added a small set of keywords as reserved ones and these keywords are reserved in all the ANSI SQL standards (SQL-92, SQL-99, SQL-2003, SQL-2008, SQL-2011, and SQL-2016) and PostgreSQL. This is because there is room to discuss which keyword should be reserved or not, .e.g., interval units (day, hour, minute, second, ...) are reserved in the ANSI SQL standards though, they are not reserved in PostgreSQL. Therefore, we need more researches about the other database-like systems (e.g., Oracle Databases, DB2, SQL server) in follow-up activities.

References:
 - The reserved/non-reserved SQL keywords in the ANSI SQL standards: https://developer.mimer.com/wp-content/uploads/2018/05/Standard-SQL-Reserved-Words-Summary.pdf
 - SQL Key Words in PostgreSQL: https://www.postgresql.org/docs/current/sql-keywords-appendix.html

## How was this patch tested?
Added tests in `TableIdentifierParserSuite`.

Closes #23259 from maropu/SPARK-26215-WIP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-02-23 08:38:47 +09:00
Dongjoon Hyun ffef3d4074 [SPARK-26950][SQL][TEST] Make RandomDataGenerator use Float.NaN or Double.NaN for all NaN values
## What changes were proposed in this pull request?

Apache Spark uses the predefined `Float.NaN` and `Double.NaN` for NaN values, but there exists more NaN values with different binary presentations.

```scala
scala> java.nio.ByteBuffer.allocate(4).putFloat(Float.NaN).array
res1: Array[Byte] = Array(127, -64, 0, 0)

scala> val x = java.lang.Float.intBitsToFloat(-6966608)
x: Float = NaN

scala> java.nio.ByteBuffer.allocate(4).putFloat(x).array
res2: Array[Byte] = Array(-1, -107, -78, -80)
```

Since users can have these values, `RandomDataGenerator` generates these NaN values. However, this causes `checkEvaluationWithUnsafeProjection` failures due to the difference between `UnsafeRow` binary presentation. The following is the UT failure instance. This PR aims to fix this UT flakiness.

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102528/testReport/

## How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #23851 from dongjoon-hyun/SPARK-26950.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-22 12:25:26 +08:00
Shixiong Zhu 77b99af573
[SPARK-26824][SS] Fix the checkpoint location and _spark_metadata when it contains special chars
## What changes were proposed in this pull request?

When a user specifies a checkpoint location or a file sink output using a path containing special chars that need to be escaped in a path, the streaming query will store checkpoint and file sink metadata in a wrong place. In this PR, I uploaded a checkpoint that was generated by the following codes using Spark 2.4.0 to show this issue:

```
implicit val s = spark.sqlContext
val input = org.apache.spark.sql.execution.streaming.MemoryStream[Int]
input.addData(1, 2, 3)
val q = input.toDF.writeStream.format("parquet").option("checkpointLocation", ".../chk %#chk").start(".../output %#output")
q.stop()
```
Here is the structure of the directory:
```
sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0
├── chk%252520%252525%252523chk
│   ├── commits
│   │   └── 0
│   ├── metadata
│   └── offsets
│       └── 0
├── output %#output
│   └── part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet
└── output%20%25%23output
    └── _spark_metadata
        └── 0
```

In this checkpoint, the user specified checkpoint location is `.../chk %#chk` but the real path to store the checkpoint is `.../chk%252520%252525%252523chk` (this is generated by escaping the original path three times). The user specified output path is `.../output %#output` but the path to store `_spark_metadata` is `.../output%20%25%23output/_spark_metadata` (this is generated by escaping the original path once). The data files are still in the correct path (such as `.../output %#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet`).

This checkpoint will be used in unit tests in this PR.

The fix is just simply removing improper `Path.toUri` calls to fix the issue.

However, as the user may not read the release note and is not aware of this checkpoint location change, if they upgrade Spark without moving checkpoint to the new location, their query will just start from the scratch. In order to not surprise the users, this PR also adds a check to **detect the impacted paths and throws an error** to include the migration guide. This check can be turned off by an internal sql conf `spark.sql.streaming.checkpoint.escapedPathCheck.enabled`. Here are examples of errors that will be reported:

- Streaming checkpoint error:
```
Error: we detected a possible problem with the location of your checkpoint and you
likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out checkpoints for
structured streaming. While this was corrected in Spark 3.0, it appears that your
query was started using an earlier version that incorrectly handled the checkpoint
path.

Correct Checkpoint Directory: /.../chk %#chk
Incorrect Checkpoint Directory: /.../chk%252520%252525%252523chk

Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

- File sink error (`_spark_metadata`):
```
Error: we detected a possible problem with the location of your "_spark_metadata"
directory and you likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out the
"_spark_metadata" directory for structured streaming. While this was corrected in
Spark 3.0, it appears that your query was started using an earlier version that
incorrectly handled the "_spark_metadata" path.

Correct "_spark_metadata" Directory: /.../output %#output/_spark_metadata
Incorrect "_spark_metadata" Directory: /.../output%20%25%23output/_spark_metadata

Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

## How was this patch tested?

The new unit tests.

Closes #23733 from zsxwing/path-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-02-20 15:44:20 -08:00
Maxim Gekk 331ac60f28 [SPARK-26900][SQL] Simplify truncation to quarter of year
## What changes were proposed in this pull request?

In the PR, I propose to simplify timestamp truncation to quarter of year by using *java.time* API directly. The `LocalDate` instance can be truncation to quarter timestamp via adjusting by chrono field `IsoFields.DAY_OF_QUARTER`.

## How was this patch tested?

This was checked by existing test suite - `DateTimeUtilsSuite`.

Closes #23808 from MaxGekk/date-quarter-of-year.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-20 08:55:08 -06:00
Hyukjin Kwon ab850c02f7 [SPARK-26901][SQL][R] Adds child's output into references to avoid column-pruning for vectorized gapply()
## What changes were proposed in this pull request?

Currently, looks column pruning is done to vectorized `gapply()`. Given R native function could use all referred fields so it shouldn't be pruned. To avoid this, it adds child's output into `references` like `OutputConsumer`.

```
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
explain(count(groupBy(gapply(df,
                             "gear",
                             function(key, group) {
                               data.frame(gear = key[[1]], disp = mean(group$disp))
                             },
                             structType("gear double, disp double")))), TRUE)
```

**Before:**

```
== Optimized Logical Plan ==
Aggregate [count(1) AS count#41L]
+- Project
   +- FlatMapGroupsInRWithArrow [...]
      +- Project [gear#9]
         +- LogicalRDD [mpg#0, cyl#1, disp#2, hp#3, drat#4, wt#5, qsec#6, vs#7, am#8, gear#9, carb#10], false

== Physical Plan ==
*(4) HashAggregate(keys=[], functions=[count(1)], output=[count#41L])
+- Exchange SinglePartition
   +- *(3) HashAggregate(keys=[], functions=[partial_count(1)], output=[count#44L])
      +- *(3) Project
         +- FlatMapGroupsInRWithArrow [...]
            +- *(2) Sort [gear#9 ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(gear#9, 200)
                  +- *(1) Project [gear#9]
                     +- *(1) Scan ExistingRDD arrow[mpg#0,cyl#1,disp#2,hp#3,drat#4,wt#5,qsec#6,vs#7,am#8,gear#9,carb#10]
```

**After:**

```
== Optimized Logical Plan ==
Aggregate [count(1) AS count#91L]
+- Project
   +- FlatMapGroupsInRWithArrow [...]
      +- LogicalRDD [mpg#0, cyl#1, disp#2, hp#3, drat#4, wt#5, qsec#6, vs#7, am#8, gear#9, carb#10], false

== Physical Plan ==
*(4) HashAggregate(keys=[], functions=[count(1)], output=[count#91L])
+- Exchange SinglePartition
   +- *(3) HashAggregate(keys=[], functions=[partial_count(1)], output=[count#94L])
      +- *(3) Project
         +- FlatMapGroupsInRWithArrow [...]
            +- *(2) Sort [gear#9 ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(gear#9, 200)
                  +- *(1) Scan ExistingRDD arrow[mpg#0,cyl#1,disp#2,hp#3,drat#4,wt#5,qsec#6,vs#7,am#8,gear#9,carb#10]
```

Currently, it adds corrupt values for missing columns (via pruned columnar batches to Arrow writers that requires non-pruned columns) such as:

```r
...
  c(7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 0, 0, 4.17777978645388e-314)
  c(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1.04669129845114e+219)
  c(3.4482690635875e-313, 3.4482690635875e-313, 3.4482690635875e-313,
  c(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2.47032822920623e-323)
...
```

which should be something like:

```r
...
  c(4, 4, 1, 2, 2, 4, 4, 1, 2, 1, 1, 2)
  c(26, 30.4, 15.8, 19.7, 15)
  c(4, 4, 8, 6, 8)
  c(120.3, 95.1, 351, 145, 301)
...
```

## How was this patch tested?

Manually tested, and unit tests were added.

The test code is basiaclly:

```r
df <- createDataFrame(mtcars)
count(gapply(df,
             c("gear"),
             function(key, group) {
                stopifnot(all(group$hp > 50))
                group
             },
             schema(df)))
```

`mtcars`'s hp is all more then 50.

```r
> mtcars$hp > 50
 [1] TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE
[16] TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE
[31] TRUE TRUE
```

However, due to corrpt value, (like 0 or 7.xxxxx), werid values were found. So, it's currently being failed as below in the master:

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 82 in stage 1.0 failed 1 times, most recent failure: Lost task 82.0 in stage 1.0 (TID 198, localhost, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (crashed)
 Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
```

I also compared the total length while I am here. Regular `gapply` without Arrow has some holes .. so I had to compare the results with R data frame.

Closes #23810 from HyukjinKwon/SPARK-26901.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-20 10:24:40 +08:00
Ryan Blue 60caa92dea [SPARK-26666][SQL] Support DSv2 overwrite and dynamic partition overwrite.
## What changes were proposed in this pull request?

This adds two logical plans that implement the ReplaceData operation from the [logical plans SPIP](https://docs.google.com/document/d/1gYm5Ji2Mge3QBdOliFV5gSPTKlX4q1DCBXIkiyMv62A/edit?ts=5a987801#heading=h.m45webtwxf2d). These two plans will be used to implement Spark's `INSERT OVERWRITE` behavior for v2.

Specific changes:
* Add `SupportsTruncate`, `SupportsOverwrite`, and `SupportsDynamicOverwrite` to DSv2 write API
* Add `OverwriteByExpression` and `OverwritePartitionsDynamic` plans (logical and physical)
* Add new plans to DSv2 write validation rule `ResolveOutputRelation`
* Refactor `WriteToDataSourceV2Exec` into trait used by all DSv2 write exec nodes

## How was this patch tested?

* The v2 analysis suite has been updated to validate the new overwrite plans
* The analysis suite for `OverwriteByExpression` checks that the delete expression is resolved using the table's columns
* Existing tests validate that overwrite exec plan works
* Updated existing v2 test because schema is used to validate overwrite

Closes #23606 from rdblue/SPARK-26666-add-overwrite.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-18 13:16:28 +08:00
maryannxue a7e3da42cd [SPARK-26840][SQL] Avoid cost-based join reorder in presence of join hints
## What changes were proposed in this pull request?

This is a fix for https://github.com/apache/spark/pull/23524, which did not stop cost-based join reorder when the CostBasedJoinReorder rule recurses down the tree and applies join reorder for nested joins with hints.

The issue had not been detected by the existing tests because CBO is disabled by default.

## How was this patch tested?

Enabled CBO for JoinHintSuite.

Closes #23759 from maryannxue/spark-26840.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-02-14 16:56:55 -08:00
Wenchen Fan 974f524992 [SPARK-26798][SQL] HandleNullInputsForUDF should trust nullability
## What changes were proposed in this pull request?

There is a very old TODO in `HandleNullInputsForUDF`, saying that we can skip the null check if input is not nullable. We leverage the nullability info at many places, we can trust it here too.

## How was this patch tested?

re-enable an ignored test

Closes #23712 from cloud-fan/minor.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-02-14 00:22:11 +09:00
Dilip Biswal 7f44c9a252 [SPARK-26864][SQL] Query may return incorrect result when python udf is used as a join condition and the udf uses attributes from both legs of left semi join.
## What changes were proposed in this pull request?
In SPARK-25314, we supported the scenario of having a python UDF that refers to attributes from both legs of a join condition by rewriting the plan to convert an inner join or left semi join to a filter over a cross join. In case of left semi join, this transformation may cause incorrect results when the right leg of join condition produces duplicate rows based on the join condition. This fix disallows the rewrite for left semi join and raises an error in the case like we do for other types of join. In future, we should have separate rule in optimizer to convert left semi join to inner join (I am aware of one case we could do it if we leverage informational constraint i.e when we know the right side does not produce duplicates).

**Python**

```SQL
>>> from pyspark import SparkContext
>>> from pyspark.sql import SparkSession, Column, Row
>>> from pyspark.sql.functions import UserDefinedFunction, udf
>>> from pyspark.sql.types import *
>>> from pyspark.sql.utils import AnalysisException
>>>
>>> spark.conf.set("spark.sql.crossJoin.enabled", "True")
>>> left = spark.createDataFrame([Row(lc1=1, lc2=1), Row(lc1=2, lc2=2)])
>>> right = spark.createDataFrame([Row(rc1=1, rc2=1), Row(rc1=1, rc2=1)])
>>> func = udf(lambda a, b: a == b, BooleanType())
>>> df = left.join(right, func("lc1", "rc1"), "leftsemi").show()
19/02/12 16:07:10 WARN PullOutPythonUDFInJoinCondition: The join condition:<lambda>(lc1#0L, rc1#4L) of the join plan contains PythonUDF only, it will be moved out and the join plan will be turned to cross join.
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
|  1|  1|
+---+---+
```

**Scala**

```SQL
scala> val left = Seq((1, 1), (2, 2)).toDF("lc1", "lc2")
left: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> val right = Seq((1, 1), (1, 1)).toDF("rc1", "rc2")
right: org.apache.spark.sql.DataFrame = [rc1: int, rc2: int]

scala> val equal = udf((p1: Integer, p2: Integer) => {
     |   p1 == p2
     | })
equal: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$2141/11016292394666f1b5,BooleanType,List(Some(Schema(IntegerType,true)), Some(Schema(IntegerType,true))),None,false,true)

scala> val df = left.join(right, equal(col("lc1"), col("rc1")), "leftsemi")
df: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> df.show()
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
+---+---+

```

## How was this patch tested?
Modified existing tests.

Closes #23769 from dilipbiswal/dkb_python_udf_in_join.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-13 21:14:19 +08:00
Hyukjin Kwon 8126d09fb5 [SPARK-26761][SQL][R] Vectorized R gapply() implementation
## What changes were proposed in this pull request?

This PR targets to add vectorized `gapply()` in R, Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
collect(gapply(df,
               "gear",
               function(key, group) {
                 data.frame(gear = key[[1]], disp = mean(group$disp) > group$disp)
               },
               structType("gear double, disp boolean")))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

**R code**

```r
rdf <- read.csv("500000.csv")
rdf <- rdf[, c("Month.of.Joining", "Weight.in.Kgs.")]  # We're only interested in the key and values to calculate.
df <- cache(createDataFrame(rdf))
count(df)

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  count(gapply(df,
               "Month_of_Joining",
               function(key, group) {
                 data.frame(Month_of_Joining = key[[1]], Weight_in_Kgs_ = mean(group$Weight_in_Kgs_) > group$Weight_in_Kgs_)
               },
               structType("Month_of_Joining integer, Weight_in_Kgs_ boolean")))
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 35.67459 secs
```

```
Time difference of 4.301399 secs
```

The performance improvement was around **829%**.

**Note that** I am 100% sure this PR improves more then 829% because I gave up testing it with non-Arrow optimization because it took super super super long when the data size becomes bigger.

### Limitations

- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.

- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.

## How was this patch tested?

Unit tests were added

**TODOs:**
- [x] Draft codes
- [x] make the tests passed
- [x] make the CRAN check pass
- [x] Performance measurement
- [x] Supportability investigation (for instance types)

Closes #23746 from HyukjinKwon/SPARK-26759.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-13 11:19:58 +08:00
Dilip Biswal 5a7403623d [SPARK-26853][SQL] Add example and version for commonly used aggregate function descriptions
## What changes were proposed in this pull request?
This improves the expression description for commonly used aggregate functions such as Max, Min, Count, etc.

## How was this patch tested?
Verified the function description manually from the shell.

Closes #23756 from dilipbiswal/dkb_expr_description_aggregate.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-02-11 23:24:54 -08:00
Maxim Gekk 9c6efd0427 [SPARK-26740][SPARK-26654][SQL] Make statistics of timestamp/date columns independent from system time zones
## What changes were proposed in this pull request?

In the PR, I propose to covert underlying types of timestamp/date columns to strings, and store the converted values as column statistics. This makes statistics for timestamp/date columns independent from system time zone while saving and retrieving such statistics.

I bumped versions of stored statistics from 1 to 2 since the PR changes the format.

## How was this patch tested?

The changes were tested by `StatisticsCollectionSuite` and by `StatisticsSuite`.

Closes #23662 from MaxGekk/column-stats-time-date.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-12 10:58:00 +08:00
Gabor Somogyi 701b06a7e2 [SPARK-26389][SS] Add force delete temp checkpoint configuration
## What changes were proposed in this pull request?

Not all users wants to keep temporary checkpoint directories. Additionally hard to restore from it.

In this PR I've added a force delete flag which is default `false`. Additionally not clear for users when temporary checkpoint directory deleted so added log messages to explain this a bit more.

## How was this patch tested?

Existing + additional unit tests.

Closes #23732 from gaborgsomogyi/SPARK-26389.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-02-08 10:22:51 -08:00
Maxim Gekk 96c6c295cc [SPARK-26805][SQL] Eliminate double checking of stringToDate and stringToTimestamp inputs
## What changes were proposed in this pull request?

In the PR, I propose to eliminate checking of parsed segments inside of the `stringToDate` and `stringToTimestamp` because such checking is already performed while constructing *java.time* classes, in particular inside of `LocalDate` and `LocalTime`. As a consequence of removing the explicit checks, the `isInvalidDate` method is not needed any more, and it was removed from `DateTimeUtils`.

## How was this patch tested?

This was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite` and `CastSuite`.

Closes #23717 from MaxGekk/datetimeutils-refactoring.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-02 18:20:16 -06:00
wuyi 8f968b4c06 [SPARK-26730][SQL] Strip redundant AssertNotNull for ExpressionEncoder's serializer
## What changes were proposed in this pull request?

For types like Product, we've already add AssertNotNull when we construct serializer(see code below), so we could strip redundant AssertNotNull for those types.

```
val fieldValue = Invoke(
    AssertNotNull(inputObject, walkedTypePath), fieldName, dataTypeFor(fieldType),
    returnNullable = !fieldType.typeSymbol.asClass.isPrimitive)
```
## How was this patch tested?

Existed.

Closes #23651 from Ngone51/dev-strip-redundant-assertnotnull-for-ecnoder-serializer.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-01 10:48:37 +08:00
Gengliang Wang df4c53e44b [SPARK-26673][SQL] File source V2 writes: create framework and migrate ORC
## What changes were proposed in this pull request?

Create a framework for write path of File Source V2.
Also, migrate write path of ORC to V2.

Supported:
* Write to file as Dataframe

Not Supported:
* Partitioning, which is still under development in the data source V2 project.
* Bucketing, which is still under development in the data source V2 project.
* Catalog.

## How was this patch tested?

Unit test

Closes #23601 from gengliangwang/orc_write.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 21:29:01 +08:00
Wenchen Fan 0e2c487459 [SPARK-26448][SQL][FOLLOWUP] should not normalize grouping expressions for final aggregate
## What changes were proposed in this pull request?

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

`AggUtils.createAggregate` is not the right place to normalize the grouping expressions, as final aggregate is also created by it. The grouping expressions of final aggregate should be attributes which refer to the grouping expressions in partial aggregate.

This PR moves the normalization to the caller side of `AggUtils`.

## How was this patch tested?

existing tests

Closes #23692 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 16:20:18 +08:00
Hyukjin Kwon d4d6df2f7d [SPARK-26745][SQL] Revert count optimization in JSON datasource by SPARK-24959
## What changes were proposed in this pull request?

This PR reverts JSON count optimization part of #21909.

We cannot distinguish the cases below without parsing:

```
[{...}, {...}]
```

```
[]
```

```
{...}
```

```bash
# empty string
```

when we `count()`. One line (input: IN) can be, 0 record, 1 record and multiple records and this is dependent on each input.

See also https://github.com/apache/spark/pull/23665#discussion_r251276720.

## How was this patch tested?

Manually tested.

Closes #23667 from HyukjinKwon/revert-SPARK-24959.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-31 14:32:31 +08:00
Bruce Robbins 7781c6fd73 [SPARK-26378][SQL] Restore performance of queries against wide CSV/JSON tables
## What changes were proposed in this pull request?

After [recent changes](11e5f1bcd4) to CSV parsing to return partial results for bad CSV records, queries of wide CSV tables slowed considerably. That recent change resulted in every row being recreated, even when the associated input record had no parsing issues and the user specified no corrupt record field in his/her schema.

The change to FailureSafeParser.scala also impacted queries against wide JSON tables as well.

In this PR, I propose that a row should be recreated only if columns need to be shifted due to the existence of a corrupt column field in the user-supplied schema. Otherwise, the code should use the row as-is (For CSV input, it will have values for the columns that could be converted, and also null values for columns that could not be converted).

See benchmarks below. The CSV benchmark for 1000 columns went from 120144 ms to 89069 ms, a savings of 25% (this only brings the cost down to baseline levels. Again, see benchmarks below).

Similarly, the JSON benchmark for 1000 columns (added in this PR) went from 109621 ms to 80871 ms, also a savings of 25%.

Still, partial results functionality is preserved:

<pre>
bash-3.2$ cat test2.csv
"hello",1999-08-01,"last"
"there","bad date","field"
"again","2017-11-22","in file"
bash-3.2$ bin/spark-shell
...etc...
scala> val df = spark.read.schema("a string, b date, c string").csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [a: string, b: date ... 1 more field]
scala> df.show
+-----+----------+-------+
|    a|         b|      c|
+-----+----------+-------+
|hello|1999-08-01|   last|
|there|      null|  field|
|again|2017-11-22|in file|
+-----+----------+-------+
scala> val df = spark.read.schema("badRecord string, a string, b date, c string").
     | option("columnNameOfCorruptRecord", "badRecord").
     | csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [badRecord: string, a: string ... 2 more fields]
scala> df.show
+--------------------+-----+----------+-------+
|           badRecord|    a|         b|      c|
+--------------------+-----+----------+-------+
|                null|hello|1999-08-01|   last|
|"there","bad date...|there|      null|  field|
|                null|again|2017-11-22|in file|
+--------------------+-----+----------+-------+
scala>
</pre>

### CSVBenchmark Benchmarks:

baseline = commit before partial results change
PR = this PR
master = master branch

[baseline_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697109/baseline_CSVBenchmark-results.txt)
[pr_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697110/pr_CSVBenchmark-results.txt)
[master_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697111/master_CSVBenchmark-results.txt)

### JSONBenchmark Benchmarks:

baseline = commit before partial results change
PR = this PR
master = master branch

[baseline_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711040/baseline_JSONBenchmark-results.txt)
[pr_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711041/pr_JSONBenchmark-results.txt)
[master_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711042/master_JSONBenchmark-results.txt)

## How was this patch tested?

- All SQL unit tests.
- Added 2 CSV benchmarks
- Python core and SQL tests

Closes #23336 from bersprockets/csv-wide-row-opt2.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-30 15:15:29 +08:00
Takeshi Yamamuro 92706e6576
[SPARK-26747][SQL] Makes GetMapValue nullability more precise
## What changes were proposed in this pull request?
In master, `GetMapValue` nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L371)

But, If input key is foldable, we could make its nullability more precise.
This fix is the same with SPARK-26637(#23566).

## How was this patch tested?
Added tests in `ComplexTypeSuite`.

Closes #23669 from maropu/SPARK-26747.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-28 13:39:50 -08:00
Maxim Gekk 58e42cf506 [SPARK-26719][SQL] Get rid of java.util.Calendar in DateTimeUtils
## What changes were proposed in this pull request?

- Replacing `java.util.Calendar` in  `DateTimeUtils. truncTimestamp` and in `DateTimeUtils.getOffsetFromLocalMillis ` by equivalent code using Java 8 API for timestamp manipulations. The reason is `java.util.Calendar` is based on the hybrid calendar (Julian+Gregorian) but *java.time* classes use Proleptic Gregorian calendar which assumes by SQL standard.
-  Replacing `Calendar.getInstance()` in `DateTimeUtilsSuite` by similar code in `DateTimeTestUtils` using *java.time* classes

## How was this patch tested?

The changes were tested by existing suites: `DateExpressionsSuite`, `DateFunctionsSuite` and `DateTimeUtilsSuite`.

Closes #23641 from MaxGekk/cleanup-date-time-utils.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-28 10:52:17 -06:00
Kris Mok 860336d31e [SPARK-26735][SQL] Verify plan integrity for special expressions
## What changes were proposed in this pull request?

Add verification of plan integrity with regards to special expressions being hosted only in supported operators. Specifically:

- `AggregateExpression`: should only be hosted in `Aggregate`, or indirectly in `Window`
- `WindowExpression`: should only be hosted in `Window`
- `Generator`: should only be hosted in `Generate`

This will help us catch errors in future optimizer rules that incorrectly hoist special expression out of their supported operator.

TODO: This PR actually caught a bug in the analyzer in the test case `SPARK-23957 Remove redundant sort from subquery plan(scalar subquery)` in `SubquerySuite`, where a `max()` aggregate function is hosted in a `Sort` operator in the analyzed plan, which is invalid. That test case is disabled in this PR.
SPARK-26741 has been opened to track the fix in the analyzer.

## How was this patch tested?

Added new test case in `OptimizerStructuralIntegrityCheckerSuite`

Closes #23658 from rednaxelafx/plan-integrity.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-26 22:26:10 -08:00
hyukjinkwon e8982ca7ad [SPARK-25981][R] Enables Arrow optimization from R DataFrame to Spark DataFrame
## What changes were proposed in this pull request?

This PR targets to support Arrow optimization for conversion from R DataFrame to Spark DataFrame.
Like PySpark side, it falls back to non-optimization code path when it's unable to use Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
collect(createDataFrame(mtcars))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

**R code**

```r
createDataFrame(mtcars) # Initializes
rdf <- read.csv("500000.csv")

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  createDataFrame(rdf)
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 29.9468 secs
```

```
Time difference of 3.222129 secs
```

The performance improvement was around **950%**.
Actually, this PR improves around **1200%**+ because this PR includes a small optimization about regular R DataFrame -> Spark DatFrame. See https://github.com/apache/spark/pull/22954#discussion_r231847272

### Limitations:

For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.
In this case, we decide to fall back to non-optimization code path.

## How was this patch tested?

Small test was added.

I manually forced to set this optimization `true` for _all_ R tests and they were _all_ passed (with few of fallback warnings).

**TODOs:**
- [x] Draft codes
- [x] make the tests passed
- [x] make the CRAN check pass
- [x] Performance measurement
- [x] Supportability investigation (for instance types)
- [x] Wait for Arrow 0.12.0 release
- [x] Fix and match it to Arrow 0.12.0

Closes #22954 from HyukjinKwon/r-arrow-createdataframe.

Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-27 10:45:49 +08:00
SongYadong aa3d16d68b [SPARK-26698][CORE] Use ConfigEntry for hardcoded configs for memory and storage categories
## What changes were proposed in this pull request?

This PR makes hardcoded configs about spark memory and storage to use `ConfigEntry` and put them in the config package.

## How was this patch tested?

Existing unit tests.

Closes #23623 from SongYadong/configEntry_for_mem_storage.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-25 22:28:12 -06:00
Bruce Robbins f17a3d9c3a
[SPARK-26711][SQL] Lazily convert string values to BigDecimal during JSON schema inference
## What changes were proposed in this pull request?

This PR fixes a bug where JSON schema inference attempts to convert every String value to a BigDecimal regardless of the setting of "prefersDecimal". With that bug, behavior is still correct, but performance is impacted.

This PR makes this conversion lazy, so it is only performed if prefersDecimal is set to true.

Using Spark with a single executor thread to infer the schema of a single-column, 100M row JSON file, the performance impact is as follows:

option | baseline | pr
-----|----|-----
inferTimestamp=_default_<br>prefersDecimal=_default_ | 12.5 minutes | 6.1 minutes |
inferTimestamp=false<br>prefersDecimal=_default_ | 6.5 minutes | 49 seconds |
inferTimestamp=false<br>prefersDecimal=true | 6.5 minutes | 6.5 minutes |

## How was this patch tested?

I ran JsonInferSchemaSuite and JsonSuite. Also, I ran manual tests to see performance impact (see above).

Closes #23653 from bersprockets/SPARK-26711_improved.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-25 16:14:38 -08:00
Maxim Gekk e3411a82c3 [SPARK-26720][SQL] Remove DateTimeUtils methods based on system default time zone
## What changes were proposed in this pull request?

In the PR, I propose to remove the following methods from `DateTimeUtils`:
- `timestampAddInterval` and `stringToTimestamp` - used only in test suites
- `truncTimestamp`, `getSeconds`, `getMinutes`, `getHours` - those methods assume system default time zone. They are not used in Spark.

## How was this patch tested?

This was tested by `DateTimeUtilsSuite` and `UnsafeArraySuite`.

Closes #23643 from MaxGekk/unused-date-time-utils.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-25 17:06:22 +08:00
Gengliang Wang f5b9370da2 [SPARK-26709][SQL] OptimizeMetadataOnlyQuery does not handle empty records correctly
## What changes were proposed in this pull request?

When reading from empty tables, the optimization `OptimizeMetadataOnlyQuery` may return wrong results:
```
sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)")
sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)")
sql("SELECT MAX(p1) FROM t")
```
The result is supposed to be `null`. However, with the optimization the result is `5`.

The rule is originally ported from https://issues.apache.org/jira/browse/HIVE-1003 in #13494. In Hive, the rule is disabled by default in a later release(https://issues.apache.org/jira/browse/HIVE-15397), due to the same problem.

It is hard to completely avoid the correctness issue. Because data sources like Parquet can be metadata-only. Spark can't tell whether it is empty or not without actually reading it. This PR disable the optimization by default.

## How was this patch tested?

Unit test

Closes #23635 from gengliangwang/optimizeMetadata.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-24 18:24:49 -08:00
Anton Okolnychyi 0df29bfbdc
[SPARK-26706][SQL] Fix illegalNumericPrecedence for ByteType
## What changes were proposed in this pull request?

This PR contains a minor change in `Cast$mayTruncate` that fixes its logic for bytes.

Right now, `mayTruncate(ByteType, LongType)` returns `false` while `mayTruncate(ShortType, LongType)` returns `true`. Consequently, `spark.range(1, 3).as[Byte]` and `spark.range(1, 3).as[Short]` behave differently.

Potentially, this bug can silently corrupt someone's data.
```scala
// executes silently even though Long is converted into Byte
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte]
  .map(b => b - 1)
  .show()
+-----+
|value|
+-----+
|  -12|
|  -11|
|  -10|
|   -9|
|   -8|
|   -7|
|   -6|
|   -5|
|   -4|
|   -3|
+-----+
// throws an AnalysisException: Cannot up cast `id` from bigint to smallint as it may truncate
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Short]
  .map(s => s - 1)
  .show()
```
## How was this patch tested?

This PR comes with a set of unit tests.

Closes #23632 from aokolnychyi/cast-fix.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-01-24 00:12:26 +00:00
Ryan Blue d008e23ab5 [SPARK-26681][SQL] Support Ammonite inner-class scopes.
## What changes were proposed in this pull request?

This adds a new pattern to recognize Ammonite REPL classes and return the correct scope.

## How was this patch tested?

Manually tested with Spark in an Ammonite session.

Closes #23607 from rdblue/SPARK-26681-support-ammonite-scopes.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-23 08:50:03 -06:00
Takeshi Yamamuro 1ed1b4d8e1 [SPARK-26637][SQL] Makes GetArrayItem nullability more precise
## What changes were proposed in this pull request?
In the master, GetArrayItem nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L236)

But, If input array size is constant and ordinal is foldable, we could make GetArrayItem nullability more precise. This pr added code to make `GetArrayItem` nullability more precise.

## How was this patch tested?
Added tests in `ComplexTypeSuite`.

Closes #23566 from maropu/GetArrayItemNullability.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-23 15:33:02 +08:00
Maxim Gekk 64ce1c9f93 [SPARK-26657][SQL] Use Proleptic Gregorian calendar in DayWeek and in WeekOfYear
## What changes were proposed in this pull request?

The expressions `DayWeek`, `DayOfWeek`, `WeekDay` and `WeekOfYear` are changed to use Proleptic Gregorian calendar instead of the hybrid one (Julian+Gregorian). This was achieved by using Java 8 API for date/timestamp manipulation, in particular the `LocalDate` class.

Week of year calculation is performed according to ISO-8601. The first week of a week-based-year is the first Monday-based week of the standard ISO year that has at least 4 days in the new year (see https://docs.oracle.com/javase/8/docs/api/java/time/temporal/IsoFields.html).

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #23594 from MaxGekk/dayweek-gregorian.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-22 17:33:29 +01:00
Liang-Chi Hsieh f92d276653 [SPARK-25811][PYSPARK] Raise a proper error when unsafe cast is detected by PyArrow
## What changes were proposed in this pull request?

Since 0.11.0, PyArrow supports to raise an error for unsafe cast ([PR](https://github.com/apache/arrow/pull/2504)). We should use it to raise a proper error for pandas udf users when such cast is detected.

Added a SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion` to disable Arrow safe type check.

## How was this patch tested?

Added test and manually test.

Closes #22807 from viirya/SPARK-25811.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 14:54:41 +08:00
Maxim Gekk 4c1cd809f8 [SPARK-26652][SQL] Remove fromJSON and fromString from Literal
## What changes were proposed in this pull request?

The `fromString` and `fromJSON` methods of the `Literal` object are removed because they are not used.

Closes #23596

Closes #23603 from MaxGekk/remove-literal-fromstring.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 02:24:12 +08:00
Maxim Gekk 34db5f5652 [SPARK-26618][SQL] Make typed Timestamp/Date literals consistent to casting
## What changes were proposed in this pull request?

In the PR, I propose to make creation of typed Literals `TIMESTAMP` and `DATE` consistent to the `Cast` expression. More precisely, reusing the `Cast` expression in the type constructors. In this way, it allows:
- To use the same calendar in parsing methods
- To support the same set of timestamp/date patterns

For example, creating timestamp literal:
```sql
SELECT TIMESTAMP '2019-01-14 20:54:00.000'
```
behaves similarly as casting the string literal:
```sql
SELECT CAST('2019-01-14 20:54:00.000' AS TIMESTAMP)
```

## How was this patch tested?

This was tested by `SQLQueryTestSuite` as well as `ExpressionParserSuite`.

Closes #23541 from MaxGekk/timestamp-date-constructors.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-18 12:47:36 +01:00
liuxian 1b575ef5d1 [SPARK-26621][CORE] Use ConfigEntry for hardcoded configs for shuffle categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.shuffle` configs to use ConfigEntry and put them in the config package.

## How was this patch tested?
Existing unit tests

Closes #23550 from 10110346/ConfigEntry_shuffle.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 12:29:17 -06:00
Maxim Gekk 6f8c0e5255 [SPARK-26593][SQL] Use Proleptic Gregorian calendar in casting UTF8String to Date/TimestampType
## What changes were proposed in this pull request?

In the PR, I propose to use *java.time* classes in `stringToDate` and `stringToTimestamp`. This switches the methods from the hybrid calendar (Gregorian+Julian) to Proleptic Gregorian calendar. And it should make the casting consistent to other Spark classes that converts textual representation of dates/timestamps to `DateType`/`TimestampType`.

## How was this patch tested?

The changes were tested by existing suites - `HashExpressionsSuite`, `CastSuite` and `DateTimeUtilsSuite`.

Closes #23512 from MaxGekk/utf8string-timestamp-parsing.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-17 17:53:00 +01:00
Gengliang Wang c0632cec04 [SPARK-23817][SQL] Create file source V2 framework and migrate ORC read path
## What changes were proposed in this pull request?
Create a framework for file source V2 based on data source V2 API.
As a good example for demonstrating the framework, this PR also migrate ORC source. This is because ORC file source supports both row scan and columnar scan, and the implementation is simpler comparing with Parquet.

Note: Currently only read path of V2 API is done, this framework and migration are only for the read path.
Supports the following scan:
- Scan ColumnarBatch
- Scan UnsafeRow
- Push down filters
- Push down required columns

Not supported( due to the limitation of data source V2 API):
- Stats metrics
- Catalog table
- Writes

## How was this patch tested?

Unit test

Closes #23383 from gengliangwang/latest_orcV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-17 23:33:29 +08:00
Liang-Chi Hsieh 8f170787d2
[SPARK-26619][SQL] Prune the unused serializers from SerializeFromObject
## What changes were proposed in this pull request?

`SerializeFromObject` now keeps all serializer expressions for domain object even when only part of output attributes are used by top plan.

We should be able to prune unused serializers from `SerializeFromObject` in such case.

## How was this patch tested?

Added tests.

Closes #23562 from viirya/SPARK-26619.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-01-16 19:16:37 +00:00
Maxim Gekk 33b5039cd3 [SPARK-25935][SQL] Allow null rows for bad records from JSON/CSV parsers
## What changes were proposed in this pull request?

This PR reverts  #22938 per discussion in #23325

Closes #23325

Closes #23543 from MaxGekk/return-nulls-from-json-parser.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-15 13:02:55 +08:00
Maxim Gekk 115fecfd84 [SPARK-26456][SQL] Cast date/timestamp to string by Date/TimestampFormatter
## What changes were proposed in this pull request?

In the PR, I propose to switch on `TimestampFormatter`/`DateFormatter` in casting dates/timestamps to strings. The changes should make the date/timestamp casting consistent to JSON/CSV datasources and time-related functions like `to_date`, `to_unix_timestamp`/`from_unixtime`.

Local formatters are moved out from `DateTimeUtils` to where they are actually used. It allows to avoid re-creation of new formatter instance per-each call. Another reason is to have separate parser for `PartitioningUtils` because default parsing pattern cannot be used (expected optional section `[.S]`).

## How was this patch tested?

It was tested by `DateTimeUtilsSuite`, `CastSuite` and `JDBC*Suite`.

Closes #23391 from MaxGekk/thread-local-date-format.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-14 21:59:25 +08:00
maryannxue 985f966b9c [SPARK-26065][FOLLOW-UP][SQL] Revert hint behavior in join reordering
## What changes were proposed in this pull request?

This is to fix a bug in #23036 that would cause a join hint to be applied on node it is not supposed to after join reordering. For example,
```
  val join = df.join(df, "id")
  val broadcasted = join.hint("broadcast")
  val join2 = join.join(broadcasted, "id").join(broadcasted, "id")
```
There should only be 2 broadcast hints on `join2`, but after join reordering there would be 4. It is because the hint application in join reordering compares the attribute set for testing relation equivalency.
Moreover, it could still be problematic even if the child relations were used in testing relation equivalency, due to the potential exprId conflict in nested self-join.

As a result, this PR simply reverts the join reorder hint behavior change introduced in #23036, which means if a join hint is present, the join node itself will not participate in the join reordering, while the sub-joins within its children still can.

## How was this patch tested?

Added new tests

Closes #23524 from maryannxue/query-hint-followup-2.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-13 15:30:45 -08:00
Bruce Robbins 09b05487b7 [SPARK-26450][SQL] Avoid rebuilding map of schema for every column in projection
## What changes were proposed in this pull request?

When creating some unsafe projections, Spark rebuilds the map of schema attributes once for each expression in the projection. Some file format readers create one unsafe projection per input file, others create one per task. ProjectExec also creates one unsafe projection per task. As a result, for wide queries on wide tables, Spark might build the map of schema attributes hundreds of thousands of times.

This PR changes two functions to reuse the same AttributeSeq instance when creating BoundReference objects for each expression in the projection. This avoids the repeated rebuilding of the map of schema attributes.

### Benchmarks

The time saved by this PR depends on size of the schema, size of the projection, number of input files (or number of file splits), number of tasks, and file format. I chose a couple of example cases.

In the following tests, I ran the query
```sql
select * from table where id1 = 1
```

Matching rows are about 0.2% of the table.

#### Orc table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
1.772306 min | 1.487267 min | 16.082943%

#### Orc table 6000 columns, 500K rows, *17* input files

baseline | pr | improvement
----|----|----
 1.656400 min | 1.423550 min | 14.057595%

#### Orc table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
0.299878 min | 0.290339 min | 3.180926%

#### Parquet table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
1.478306 min | 1.373728 min | 7.074165%

Note: The parquet reader does not create an unsafe projection. However, the filter operation in the query causes the planner to add a ProjectExec, which does create an unsafe projection for each task. So these results have nothing to do with Parquet itself.

#### Parquet table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
0.245006 min | 0.242200 min | 1.145099%

#### CSV table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
2.390117 min | 2.182778 min | 8.674844%

#### CSV table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
1.520911 min | 1.510211 min | 0.703526%

## How was this patch tested?

SQL unit tests
Python core and SQL test

Closes #23392 from bersprockets/norebuild.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-13 23:54:19 +01:00
Maxim Gekk 4ff2b94a7c [SPARK-26503][CORE][DOC][FOLLOWUP] Get rid of spark.sql.legacy.timeParser.enabled
## What changes were proposed in this pull request?

The SQL config `spark.sql.legacy.timeParser.enabled` was removed by https://github.com/apache/spark/pull/23495. The PR cleans up the SQL migration guide and the comment for `UnixTimestamp`.

Closes #23529 from MaxGekk/get-rid-off-legacy-parser-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-13 11:20:22 +08:00
Sean Owen 51a6ba0181 [SPARK-26503][CORE] Get rid of spark.sql.legacy.timeParser.enabled
## What changes were proposed in this pull request?

Per discussion in #23391 (comment) this proposes to just remove the old pre-Spark-3 time parsing behavior.

This is a rebase of https://github.com/apache/spark/pull/23411

## How was this patch tested?

Existing tests.

Closes #23495 from srowen/SPARK-26503.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-11 08:53:12 -06:00
Wenchen Fan 1f1d98c6fa [SPARK-26580][SQL] remove Scala 2.11 hack for Scala UDF
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22732 , we tried our best to keep the behavior of Scala UDF unchanged in Spark 2.4.

However, since Spark 3.0, Scala 2.12 is the default. The trick that was used to keep the behavior unchanged doesn't work with Scala 2.12.

This PR proposes to remove the Scala 2.11 hack, as it's not useful.

## How was this patch tested?

existing tests.

Closes #23498 from cloud-fan/udf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-11 14:52:13 +08:00
Dongjoon Hyun 270916f8cd
[SPARK-26584][SQL] Remove spark.sql.orc.copyBatchToSpark internal conf
## What changes were proposed in this pull request?

This PR aims to remove internal ORC configuration to simplify the code path for Spark 3.0.0. This removes the configuration `spark.sql.orc.copyBatchToSpark` and related ORC codes including tests and benchmarks.

## How was this patch tested?

Pass the Jenkins with the reduced test coverage.

Closes #23503 from dongjoon-hyun/SPARK-26584.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-10 08:42:23 -08:00