Commit graph

26481 commits

Author SHA1 Message Date
DB Tsai d0f9614760 [SPARK-30289][SQL] Partitioned by Nested Column for InMemoryTable
### What changes were proposed in this pull request?
1. `InMemoryTable` was flatting the nested columns, and then the flatten columns was used to look up the indices which is not correct.

This PR implements partitioned by nested column for `InMemoryTable`.

### Why are the changes needed?

This PR implements partitioned by nested column for `InMemoryTable`, so we can test this features in DSv2

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing unit tests and new tests.

Closes #26929 from dbtsai/addTests.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-02-14 21:46:01 +00:00
Holden Karau d273a2bb0f [SPARK-20628][CORE][K8S] Start to improve Spark decommissioning & preemption support
This PR is based on an existing/previou PR - https://github.com/apache/spark/pull/19045

### What changes were proposed in this pull request?

This changes adds a decommissioning state that we can enter when the cloud provider/scheduler lets us know we aren't going to be removed immediately but instead will be removed soon. This concept fits nicely in K8s and also with spot-instances on AWS / preemptible instances all of which we can get a notice that our host is going away. For now we simply stop scheduling jobs, in the future we could perform some kind of migration of data during scale-down, or at least stop accepting new blocks to cache.

There is a design document at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE/edit?usp=sharing

### Why are the changes needed?

With more move to preemptible multi-tenancy, serverless environments, and spot-instances better handling of node scale down is required.

### Does this PR introduce any user-facing change?

There is no API change, however an additional configuration flag is added to enable/disable this behaviour.

### How was this patch tested?

New integration tests in the Spark K8s integration testing. Extension of the AppClientSuite to test decommissioning seperate from the K8s.

Closes #26440 from holdenk/SPARK-20628-keep-track-of-nodes-which-are-going-to-be-shutdown-r4.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-02-14 12:36:52 -08:00
HyukjinKwon b343757b1b
[SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment variable to set in both executor and driver
### What changes were proposed in this pull request?

This PR address the comment at https://github.com/apache/spark/pull/26496#discussion_r379194091 and improves the migration guide to explicitly note that the legacy environment variable to set in both executor and driver.

### Why are the changes needed?

To clarify this env should be set both in driver and executors.

### Does this PR introduce any user-facing change?

Nope.

### How was this patch tested?

I checked it via md editor.

Closes #27573 from HyukjinKwon/SPARK-29748.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-14 10:18:08 -08:00
Maxim Gekk 7137a6d065 [SPARK-30766][SQL] Fix the timestamp truncation to the HOUR and DAY levels
### What changes were proposed in this pull request?
In the PR, I propose to use Java 8 time API in timestamp truncations to the levels of `HOUR` and `DAY`. The problem is in the usage of `timeZone.getOffset(millis)` in days/hours truncations where the combined calendar (Julian + Gregorian) is used underneath.

### Why are the changes needed?
The change fix wrong truncations. For example, the following truncation to hours should print `0010-01-01 01:00:00` but it outputs wrong timestamp:
```scala
Seq("0010-01-01 01:02:03.123456").toDF()
    .select($"value".cast("timestamp").as("ts"))
    .select(date_trunc("HOUR", $"ts").cast("string"))
    .show(false)
+------------------------------------+
|CAST(date_trunc(HOUR, ts) AS STRING)|
+------------------------------------+
|0010-01-01 01:30:17                 |
+------------------------------------+
```

### Does this PR introduce any user-facing change?
Yes. After the changes, the result of the example above is:
```scala
+------------------------------------+
|CAST(date_trunc(HOUR, ts) AS STRING)|
+------------------------------------+
|0010-01-01 01:00:00                 |
+------------------------------------+
```

### How was this patch tested?
- Added new test to `DateFunctionsSuite`
- By `DateExpressionsSuite` and `DateTimeUtilsSuite`

Closes #27512 from MaxGekk/fix-trunc-old-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 22:16:57 +08:00
HyukjinKwon 2a270a731a [SPARK-30810][SQL] Parses and convert a CSV Dataset having different column from 'value' in csv(dataset) API
### What changes were proposed in this pull request?

This PR fixes `DataFrameReader.csv(dataset: Dataset[String])` API to take a `Dataset[String]` originated from a column name different from `value`. This is a long-standing bug started from the very first place.

`CSVUtils.filterCommentAndEmpty` assumed the `Dataset[String]` to be originated with `value` column. This PR changes to use the first column name in the schema.

### Why are the changes needed?

For  `DataFrameReader.csv(dataset: Dataset[String])` to support any `Dataset[String]` as the signature indicates.

### Does this PR introduce any user-facing change?
Yes,

```scala
val ds = spark.range(2).selectExpr("concat('a,b,', id) AS text").as[String]
spark.read.option("header", true).option("inferSchema", true).csv(ds).show()
```

Before:

```
org.apache.spark.sql.AnalysisException: cannot resolve '`value`' given input columns: [text];;
'Filter (length(trim('value, None)) > 0)
+- Project [concat(a,b,, cast(id#0L as string)) AS text#2]
   +- Range (0, 2, step=1, splits=Some(2))
```

After:

```
+---+---+---+
|  a|  b|  0|
+---+---+---+
|  a|  b|  1|
+---+---+---+
```

### How was this patch tested?

Unittest was added.

Closes #27561 from HyukjinKwon/SPARK-30810.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 18:20:18 +08:00
yi.wu 99b8136a86 [SPARK-25990][SQL] ScriptTransformation should handle different data types correctly
### What changes were proposed in this pull request?

We should convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`.

### Why are the changes needed?

We may hit below exception without this change:

```
[info]    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.1.6, executor driver): java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to org.apache.hadoop.hive.common.type.HiveDecimal
[info]   	at org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector.getPrimitiveJavaObject(JavaHiveDecimalObjectInspector.java:55)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazyUtils.writePrimitiveUTF8(LazyUtils.java:321)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serialize(LazySimpleSerDe.java:292)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serializeField(LazySimpleSerDe.java:247)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.doSerialize(LazySimpleSerDe.java:231)
[info]   	at org.apache.hadoop.hive.serde2.AbstractEncodingAwareSerDe.serialize(AbstractEncodingAwareSerDe.java:55)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2(ScriptTransformationExec.scala:300)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2$adapted(ScriptTransformationExec.scala:281)
[info]   	at scala.collection.Iterator.foreach(Iterator.scala:941)
[info]   	at scala.collection.Iterator.foreach$(Iterator.scala:941)
[info]   	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$1(ScriptTransformationExec.scala:281)
[info]   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1932)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Added new test. But please note that this test returns different result between Hive1.2 and Hive2.3 due to `HiveDecimal` or `SerDe` difference(don't know the root cause yet).

Closes #27556 from Ngone51/script_transform.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 16:52:28 +08:00
David Toneian b2134ee73c [SPARK-30823][PYTHON][DOCS] Set %PYTHONPATH% when building PySpark documentation on Windows
This commit is published into the public domain.

### What changes were proposed in this pull request?
In analogy to `python/docs/Makefile`, which has
> export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.8.1-src.zip)

on line 10, this PR adds
> set PYTHONPATH=..;..\lib\py4j-0.10.8.1-src.zip

to `make2.bat`.

Since there is no `realpath` in default installations of Windows, I left the relative paths unresolved. Per the instructions on how to build docs, `make.bat` is supposed to be run from `python/docs` as the working directory, so this should probably not cause issues (`%BUILDDIR%` is a relative path as well.)

### Why are the changes needed?
When building the PySpark documentation on Windows, by changing directory to `python/docs` and running `make.bat` (which runs `make2.bat`), the majority of the documentation may not be built if pyspark is not in the default `%PYTHONPATH%`. Sphinx then reports that `pyspark` (and possibly dependencies) cannot be imported.

If `pyspark` is in the default `%PYTHONPATH%`, I suppose it is that version of `pyspark` – as opposed to the version found above the `python/docs` directory – that is considered when building the documentation, which may result in documentation that does not correspond to the development version one is trying to build.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Manual tests on my Windows 10 machine. Additional tests with other environments very welcome!

Closes #27569 from DavidToneian/SPARK-30823.

Authored-by: David Toneian <david@toneian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-14 13:49:11 +09:00
maryannxue 0aed77a015 [SPARK-30801][SQL] Subqueries should not be AQE-ed if main query is not
### What changes were proposed in this pull request?
This PR makes sure AQE is either enabled or disabled for the entire query, including the main query and all subqueries.
Currently there are unsupported queries by AQE, e.g., queries that contain DPP filters. We need to make sure that if the main query is unsupported, none of the sub-queries should apply AQE, otherwise it can lead to performance regressions due to missed opportunity of sub-query reuse.

### Why are the changes needed?
To get rid of potential perf regressions when AQE is turned on.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Updated DynamicPartitionPruningSuite:
1. Removed the existing workaround `withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")`
2. Added `DynamicPartitionPruningSuiteAEOn` and `DynamicPartitionPruningSuiteAEOff` to enable testing this suite with AQE on and off options
3. Added a check in `checkPartitionPruningPredicate` to verify that the subqueries are always in sync with the main query in terms of whether AQE is applied.

Closes #27554 from maryannxue/spark-30801.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 11:20:55 +08:00
David Toneian 25db8c71a2 [PYSPARK][DOCS][MINOR] Changed :func: to :attr: Sphinx roles, fixed links in documentation of Data{Frame,Stream}{Reader,Writer}
This commit is published into the public domain.

### What changes were proposed in this pull request?
This PR fixes the documentation of `DataFrameReader`, `DataFrameWriter`, `DataStreamReader`, and `DataStreamWriter`, where attributes of other classes were misrepresented as functions. Additionally, creation of hyperlinks across modules was fixed in these instances.

### Why are the changes needed?
The old state produced documentation that suggested invalid usage of PySpark objects (accessing attributes as though they were callable.)

### Does this PR introduce any user-facing change?
No, except for improved documentation.

### How was this patch tested?
No test added; documentation build runs through.

Closes #27553 from DavidToneian/docfix-DataFrameReader-DataFrameWriter-DataStreamReader-DataStreamWriter.

Authored-by: David Toneian <david@toneian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-14 11:00:35 +09:00
Xingbo Jiang fa3517cdb1 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit 57254c9719.
2020-02-13 17:43:55 -08:00
sarthfrey-db 57254c9719 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

### Does this PR introduce any user-facing change?

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-13 16:15:00 -08:00
Ali Afroozeh e2d3983de7 [SPARK-30798][SQL] Scope Session.active in QueryExecution
### What changes were proposed in this pull request?

This PR scopes `SparkSession.active` to prevent problems with processing queries with possibly different spark sessions (and different configs). A new method, `withActive` is introduced on `SparkSession` that restores the previous spark session after the block of code is executed.

### Why are the changes needed?
`SparkSession.active` is a thread local variable that points to the current thread's spark session. It is important to note that the `SQLConf.get` method depends on `SparkSession.active`. In the current implementation it is possible that `SparkSession.active` points to a different session which causes various problems. Most of these problems arise because part of the query processing is done using the configurations of a different session. For example, when creating a data frame using a new session, i.e., `session.sql("...")`, part of the data frame is constructed using the currently active spark session, which can be a different session from the one used later for processing the query.

### Does this PR introduce any user-facing change?
The `withActive` method is introduced on `SparkSession`.

### How was this patch tested?
Unit tests (to be added)

Closes #27387 from dbaliafroozeh/UseWithActiveSessionInQueryExecution.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-13 23:58:55 +01:00
Dongjoon Hyun 74cd46eb69 [SPARK-30816][K8S][TESTS] Fix dev-run-integration-tests.sh to ignore empty params
### What changes were proposed in this pull request?

This PR aims to fix `dev-run-integration-tests.sh` to ignore empty params correctly.

### Why are the changes needed?

The following script runs `mvn` integration test like the following.
```
$ resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
...
build/mvn integration-test
-f /Users/dongjoon/APACHE/spark/pom.xml
-pl resource-managers/kubernetes/integration-tests
-am
-Pscala-2.12
-Pkubernetes
-Pkubernetes-integration-tests
-Djava.version=8
-Dspark.kubernetes.test.sparkTgz=N/A
-Dspark.kubernetes.test.imageTag=N/A
-Dspark.kubernetes.test.imageRepo=docker.io/kubespark
-Dspark.kubernetes.test.deployMode=minikube
-Dtest.include.tags=k8s
-Dspark.kubernetes.test.namespace=
-Dspark.kubernetes.test.serviceAccountName=
-Dspark.kubernetes.test.kubeConfigContext=
-Dspark.kubernetes.test.master=
-Dtest.exclude.tags=
-Dspark.kubernetes.test.jvmImage=spark
-Dspark.kubernetes.test.pythonImage=spark-py
-Dspark.kubernetes.test.rImage=spark-r
```

After this PR, the empty parameters like the followings will be skipped like the original design.
```
-Dspark.kubernetes.test.namespace=
-Dspark.kubernetes.test.serviceAccountName=
-Dspark.kubernetes.test.kubeConfigContext=
-Dspark.kubernetes.test.master=
-Dtest.exclude.tags=
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Pass the Jenkins K8S integration test.

Closes #27566 from dongjoon-hyun/SPARK-30816.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-13 11:42:00 -08:00
Dongjoon Hyun 859699135c [SPARK-30807][K8S][TESTS] Support Java 11 in K8S integration tests
### What changes were proposed in this pull request?

This PR aims to support JDK11 test in K8S integration tests.
- This is an update in testing framework instead of individual tests.
- This will enable JDK11 runtime test when you didn't installed JDK11 on your local system.

### Why are the changes needed?

Apache Spark 3.0.0 adds JDK11 support, but K8s integration tests use JDK8 until now.

### Does this PR introduce any user-facing change?

No. This is a dev-only test-related PR.

### How was this patch tested?

This is irrelevant to Jenkins UT, but Jenkins K8S IT (JDK8) should pass.
- https://github.com/apache/spark/pull/27559#issuecomment-585903489 (JDK8 Passed)

And, manually do the following for JDK11 test.
```
$ NO_MANUAL=1 ./dev/make-distribution.sh --r --pip --tgz -Phadoop-3.2 -Pkubernetes
$ resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh --java-image-tag 11-jre-slim --spark-tgz $PWD/spark-*.tgz
```

```
$ docker run -it --rm kubespark/spark:1318DD8A-2B15-4A00-BC69-D0E90CED235B /usr/local/openjdk-11/bin/java --version | tail -n1
OpenJDK 64-Bit Server VM 18.9 (build 11.0.6+10, mixed mode)
```

Closes #27559 from dongjoon-hyun/SPARK-30807.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-13 11:17:27 -08:00
Wenchen Fan a4ceea6868 [SPARK-30751][SQL] Combine the skewed readers into one in AQE skew join optimizations
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
This is a followup of https://github.com/apache/spark/pull/26434

This PR use one special shuffle reader for skew join, so that we only have one join after optimization. In order to do that, this PR
1. add a very general `CustomShuffledRowRDD` which support all kind of partition arrangement.
2. move the logic of coalescing shuffle partitions to a util function, and call it during skew join optimization, to totally decouple with the `ReduceNumShufflePartitions` rule. It's too complicated to interfere skew join with `ReduceNumShufflePartitions`, as you need to consider the size of split partitions which don't respect target size already.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
The current skew join optimization has a serious performance issue: the size of the query plan depends on the number and size of skewed partitions.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
no

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
existing tests

test UI manually:
![image](https://user-images.githubusercontent.com/3182036/74357390-cfb30480-4dfa-11ea-83f6-825d1b9379ca.png)

explain output
```
AdaptiveSparkPlan(isFinalPlan=true)
+- OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$403a2ed5, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap1f
   +- *(5) SortMergeJoin(skew=true) [key1#2L], [key2#6L], Inner
      :- *(3) Sort [key1#2L ASC NULLS FIRST], false, 0
      :  +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB)
      :     +- ShuffleQueryStage 0
      :        +- Exchange hashpartitioning(key1#2L, 200), true, [id=#53]
      :           +- *(1) Project [(id#0L % 2) AS key1#2L]
      :              +- *(1) Filter isnotnull((id#0L % 2))
      :                 +- *(1) Range (0, 100000, step=1, splits=6)
      +- *(4) Sort [key2#6L ASC NULLS FIRST], false, 0
         +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB)
            +- ShuffleQueryStage 1
               +- Exchange hashpartitioning(key2#6L, 200), true, [id=#64]
                  +- *(2) Project [((id#4L % 2) + 1) AS key2#6L]
                     +- *(2) Filter isnotnull(((id#4L % 2) + 1))
                        +- *(2) Range (0, 100000, step=1, splits=6)
```

Closes #27493 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-13 20:09:24 +01:00
Takeshi Yamamuro 3c4044ea77 [SPARK-30703][SQL][DOCS] Add a document for the ANSI mode
### What changes were proposed in this pull request?

This pr intends to add a document for the ANSI mode;

<img width="600" alt="Screen Shot 2020-02-13 at 8 08 52" src="https://user-images.githubusercontent.com/692303/74386041-5934f780-4e38-11ea-8162-26e524e11c65.png">
<img width="600" alt="Screen Shot 2020-02-13 at 8 09 13" src="https://user-images.githubusercontent.com/692303/74386040-589c6100-4e38-11ea-8a64-899788eaf55f.png">
<img width="600" alt="Screen Shot 2020-02-13 at 8 09 26" src="https://user-images.githubusercontent.com/692303/74386039-5803ca80-4e38-11ea-949f-049208d2203d.png">
<img width="600" alt="Screen Shot 2020-02-13 at 8 09 38" src="https://user-images.githubusercontent.com/692303/74386036-563a0700-4e38-11ea-9ec3-87a8f6771cf0.png">

### Why are the changes needed?

For better document coverage and usability.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

N/A

Closes #27489 from maropu/SPARK-30703.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-13 10:53:55 -08:00
Liang Zhang 82d0aa37ae [SPARK-30762] Add dtype=float32 support to vector_to_array UDF
### What changes were proposed in this pull request?
In this PR, we add a parameter in the python function vector_to_array(col) that allows converting to a column of arrays of Float (32bits) in scala, which would be mapped to a numpy array of dtype=float32.

### Why are the changes needed?
In the downstream ML training, using float32 instead of float64 (default) would allow a larger batch size, i.e., allow more data to fit in the memory.

### Does this PR introduce any user-facing change?
Yes.
Old: `vector_to_array()` only take one param
```
df.select(vector_to_array("colA"), ...)
```
New: `vector_to_array()` can take an additional optional param: `dtype` = "float32" (or "float64")
```
df.select(vector_to_array("colA", "float32"), ...)
```

### How was this patch tested?
Unit test in scala.
doctest in python.

Closes #27522 from liangz1/udf-float32.

Authored-by: Liang Zhang <liang.zhang@databricks.com>
Signed-off-by: WeichenXu <weichen.xu@databricks.com>
2020-02-13 23:55:13 +08:00
Yuming Wang fb0e07b08c [SPARK-29231][SQL] Constraints should be inferred from cast equality constraint
### What changes were proposed in this pull request?

This PR add support infer constraints from cast equality constraint. For example:
```scala
scala> spark.sql("create table spark_29231_1(c1 bigint, c2 bigint)")
res0: org.apache.spark.sql.DataFrame = []

scala> spark.sql("create table spark_29231_2(c1 int, c2 bigint)")
res1: org.apache.spark.sql.DataFrame = []

scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain
== Physical Plan ==
*(2) Project [c1#5L, c2#6L]
+- *(2) BroadcastHashJoin [c1#5L], [cast(c1#7 as bigint)], Inner, BuildRight
   :- *(2) Project [c1#5L, c2#6L]
   :  +- *(2) Filter (isnotnull(c1#5L) AND (c1#5L = 1))
   :     +- *(2) ColumnarToRow
   :        +- FileScan parquet default.spark_29231_1[c1#5L,c2#6L] Batched: true, DataFilters: [isnotnull(c1#5L), (c1#5L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct<c1:bigint,c2:bigint>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#209]
      +- *(1) Project [c1#7]
         +- *(1) Filter isnotnull(c1#7)
            +- *(1) ColumnarToRow
               +- FileScan parquet default.spark_29231_2[c1#7] Batched: true, DataFilters: [isnotnull(c1#7)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct<c1:int>
```

After this PR:
```scala
scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain
== Physical Plan ==
*(2) Project [c1#0L, c2#1L]
+- *(2) BroadcastHashJoin [c1#0L], [cast(c1#2 as bigint)], Inner, BuildRight
   :- *(2) Project [c1#0L, c2#1L]
   :  +- *(2) Filter (isnotnull(c1#0L) AND (c1#0L = 1))
   :     +- *(2) ColumnarToRow
   :        +- FileScan parquet default.spark_29231_1[c1#0L,c2#1L] Batched: true, DataFilters: [isnotnull(c1#0L), (c1#0L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct<c1:bigint,c2:bigint>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#99]
      +- *(1) Project [c1#2]
         +- *(1) Filter ((cast(c1#2 as bigint) = 1) AND isnotnull(c1#2))
            +- *(1) ColumnarToRow
               +- FileScan parquet default.spark_29231_2[c1#2] Batched: true, DataFilters: [(cast(c1#2 as bigint) = 1), isnotnull(c1#2)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct<c1:int>
```

### Why are the changes needed?

Improve query performance.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Unit test.

Closes #27252 from wangyum/SPARK-29231.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 22:48:27 +08:00
beliefer 04604b9899 [SPARK-30758][SQL][TESTS] Improve bracketed comments tests
### What changes were proposed in this pull request?
Although Spark SQL support bracketed comments, but `SQLQueryTestSuite` can't treat bracketed comments well and lead to generated golden files can't display bracketed comments well.
This PR will improve the treatment of bracketed comments and add three test case in `PlanParserSuite`.
Spark SQL can't support nested bracketed comments and https://github.com/apache/spark/pull/27495 used to support it.

### Why are the changes needed?
Golden files can't display well.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
New UT.

Closes #27481 from beliefer/ansi-brancket-comments.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 22:06:24 +08:00
Terry Kim a6b4b914f2 [SPARK-30613][SQL] Support Hive style REPLACE COLUMNS syntax
### What changes were proposed in this pull request?

This PR proposes to support Hive-style `ALTER TABLE ... REPLACE COLUMNS ...` as described in https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Add/ReplaceColumns

The user now can do the following:
```SQL
CREATE TABLE t (col1 int, col2 int) USING Foo;
ALTER TABLE t REPLACE COLUMNS (col2 string COMMENT 'comment2', col3 int COMMENT 'comment3');
```
, which drops the existing columns `col1` and `col2`, and add new columns `col2` and `col3`.

### Why are the changes needed?

This is a new DDL statement. Spark currently supports the Hive-style `ALTER TABLE ... CHANGE COLUMN ...`, so this new addition can be useful.

### Does this PR introduce any user-facing change?

Yes, adding a new DDL statement.

### How was this patch tested?

More tests to be added.

Closes #27482 from imback82/replace_cols.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 20:13:36 +08:00
maryannxue 453d5261b2 [SPARK-30528][SQL] Turn off DPP subquery duplication by default
### What changes were proposed in this pull request?
This PR adds a config for Dynamic Partition Pruning subquery duplication and turns it off by default due to its potential performance regression.
When planning a DPP filter, it seeks to reuse the broadcast exchange relation if the corresponding join is a BHJ with the filter relation being on the build side, otherwise it will either opt out or plan the filter as an un-reusable subquery duplication based on the cost estimate. However, the cost estimate is not accurate and only takes into account the table scan overhead, thus adding an un-reusable subquery duplication DPP filter can sometimes cause perf regression.
This PR turns off the subquery duplication DPP filter by:
1. adding a config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly` and setting it `true` by default.
2. removing the existing meaningless config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcast` since we always want to reuse broadcast results if possible.

### Why are the changes needed?
This is to fix a potential performance regression caused by DPP.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Updated DynamicPartitionPruningSuite to test the new configuration.

Closes #27551 from maryannxue/spark-30528.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 19:32:38 +08:00
iRakson 926e3a1efe [SPARK-30790] The dataType of map() should be map<null,null>
### What changes were proposed in this pull request?

`spark.sql("select map()")` returns {}.

After these changes it will return map<null,null>

### Why are the changes needed?
After changes introduced due to #27521, it is important to maintain consistency while using map().

### Does this PR introduce any user-facing change?
Yes. Now map() will give map<null,null> instead of {}.

### How was this patch tested?
UT added. Migration guide updated as well

Closes #27542 from iRakson/SPARK-30790.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 12:23:40 +08:00
Thomas Graves 496f6ac860 [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes
### What changes were proposed in this pull request?

This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile.  Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile.

The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles.

I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents.

There are various places in the code that use executor "slots" for things.  The ResourceProfile adds functionality to keep that calculation in it.   This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there.
This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things.

This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier.   That full set of changes will come with the scheduler PR that will be after this one.

The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue.

The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well.

### Why are the changes needed?

Needed for stage level scheduling feature.

### Does this PR introduce any user-facing change?

No user facing api changes added here.

### How was this patch tested?

Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases.

Closes #27313 from tgravescs/SPARK-29148.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-12 16:45:42 -06:00
Liang-Chi Hsieh 5b76367a9d [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table
### What changes were proposed in this pull request?

This is a follow-up to the PR #26956. In #26956, the patch proposed to preserve path permission when truncating table. When setting up original ACLs, we need to set user/group/other permission as ACL entries too, otherwise if the path doesn't have default user/group/other ACL entries, ACL API will complain an error `Invalid ACL: the user, group and other entries are required.`.

 In short this change makes sure:

1. Permissions for user/group/other are always kept into ACLs to work with ACL API.
2. Other custom ACLs are still kept after TRUNCATE TABLE (#26956 did this).

### Why are the changes needed?

Without this fix, `TRUNCATE TABLE` will get an error when setting up ACLs if there is no default default user/group/other ACL entries.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Update unit test.

Manual test on dev Spark cluster.

Set ACLs for a table path without default user/group/other ACL entries:
```
hdfs dfs -setfacl --set 'user:liangchi:rwx,user::rwx,group::r--,other::r--' /user/hive/warehouse/test.db/test_truncate_table

hdfs dfs -getfacl /user/hive/warehouse/test.db/test_truncate_table
# file: /user/hive/warehouse/test.db/test_truncate_table
# owner: liangchi
# group: supergroup
user::rwx
user:liangchi:rwx
group::r--
mask::rwx
other::r--
```
Then run `sql("truncate table test.test_truncate_table")`, it works by normally truncating the table and preserve ACLs.

Closes #27548 from viirya/fix-truncate-table-permission.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-12 14:27:18 -08:00
Maxim Gekk aa0d13683c [SPARK-30760][SQL] Port millisToDays and daysToMillis on Java 8 time API
### What changes were proposed in this pull request?
In the PR, I propose to rewrite the `millisToDays` and `daysToMillis` of `DateTimeUtils` using Java 8 time API.

I removed `getOffsetFromLocalMillis` from `DateTimeUtils` because it is a private methods, and is not used anymore in Spark SQL.

### Why are the changes needed?
New implementation is based on Proleptic Gregorian calendar which has been already used by other date-time functions. This changes make `millisToDays` and `daysToMillis` consistent to rest Spark SQL API related to date & time operations.

### Does this PR introduce any user-facing change?
Yes, this might effect behavior for old dates before 1582 year.

### How was this patch tested?
By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, DateExpressionsSuite`, `SQLQuerySuite` and `HiveResultSuite`.

Closes #27494 from MaxGekk/millis-2-days-java8-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 02:31:48 +08:00
Eric Wu 5919bd3b8d [SPARK-30651][SQL] Add detailed information for Aggregate operators in EXPLAIN FORMATTED
### What changes were proposed in this pull request?
Currently `EXPLAIN FORMATTED` only report input attributes of HashAggregate/ObjectHashAggregate/SortAggregate, while `EXPLAIN EXTENDED` provides more information of Keys, Functions, etc. This PR enhanced `EXPLAIN FORMATTED` to sync with original explain behavior.

### Why are the changes needed?
The newly added `EXPLAIN FORMATTED` got less information comparing to the original `EXPLAIN EXTENDED`

### Does this PR introduce any user-facing change?
Yes, taking HashAggregate explain result as example.

**SQL**
```
EXPLAIN FORMATTED
  SELECT
    COUNT(val) + SUM(key) as TOTAL,
    COUNT(key) FILTER (WHERE val > 1)
  FROM explain_temp1;
```

**EXPLAIN EXTENDED**
```
== Physical Plan ==
*(2) HashAggregate(keys=[], functions=[count(val#6), sum(cast(key#5 as bigint)), count(key#5)], output=[TOTAL#62L, count(key) FILTER (WHERE (val > 1))#71L])
+- Exchange SinglePartition, true, [id=#89]
   +- HashAggregate(keys=[], functions=[partial_count(val#6), partial_sum(cast(key#5 as bigint)), partial_count(key#5) FILTER (WHERE (val#6 > 1))], output=[count#75L, sum#76L, count#77L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.explain_temp1[key#5,val#6] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/XXX/spark-dev/spark/spark-warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,val:int>
```

**EXPLAIN FORMATTED - BEFORE**
```
== Physical Plan ==
* HashAggregate (5)
+- Exchange (4)
   +- HashAggregate (3)
      +- * ColumnarToRow (2)
         +- Scan parquet default.explain_temp1 (1)

...
...
(5) HashAggregate [codegen id : 2]
Input: [count#91L, sum#92L, count#93L]
...
...
```

**EXPLAIN FORMATTED - AFTER**
```
== Physical Plan ==
* HashAggregate (5)
+- Exchange (4)
   +- HashAggregate (3)
      +- * ColumnarToRow (2)
         +- Scan parquet default.explain_temp1 (1)

...
...
(5) HashAggregate [codegen id : 2]
Input: [count#91L, sum#92L, count#93L]
Keys: []
Functions: [count(val#6), sum(cast(key#5 as bigint)), count(key#5)]
Results: [(count(val#6)#84L + sum(cast(key#5 as bigint))#85L) AS TOTAL#78L, count(key#5)#86L AS count(key) FILTER (WHERE (val > 1))#87L]
Output: [TOTAL#78L, count(key) FILTER (WHERE (val > 1))#87L]
...
...
```

### How was this patch tested?
Three tests added in explain.sql for HashAggregate/ObjectHashAggregate/SortAggregate.

Closes #27368 from Eric5553/ExplainFormattedAgg.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 02:00:23 +08:00
Maxim Gekk 61b1e608f0 [SPARK-30759][SQL][TESTS][FOLLOWUP] Check cache initialization in StringRegexExpression
### What changes were proposed in this pull request?
Added new test to `RegexpExpressionsSuite` which checks that `cache` of compiled pattern is set when the `right` expression (pattern in `LIKE`) is a foldable expression.

### Why are the changes needed?
To be sure that `cache` in `StringRegexExpression` is initialized for foldable patterns.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By running the added test in `RegexpExpressionsSuite`.

Closes #27547 from MaxGekk/regexp-cache-test.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 23:50:34 +08:00
Maxim Gekk c1986204e5 [SPARK-30788][SQL] Support SimpleDateFormat and FastDateFormat as legacy date/timestamp formatters
### What changes were proposed in this pull request?
In the PR, I propose to add legacy date/timestamp formatters based on `SimpleDateFormat` and `FastDateFormat`:
- `LegacyFastTimestampFormatter` - uses `FastDateFormat` and supports parsing/formatting in microsecond precision. The code was borrowed from Spark 2.4, see https://github.com/apache/spark/pull/26507 & https://github.com/apache/spark/pull/26582
- `LegacySimpleTimestampFormatter` uses `SimpleDateFormat`, and support the `lenient` mode. When the `lenient` parameter is set to `false`, the parser become much stronger in checking its input.

### Why are the changes needed?
Spark 2.4.x uses the following parsers for parsing/formatting date/timestamp strings:
- `DateTimeFormat` in CSV/JSON datasource
- `SimpleDateFormat` - is used in JDBC datasource, in partitions parsing.
- `SimpleDateFormat` in strong mode (`lenient = false`), see https://github.com/apache/spark/blob/branch-2.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala#L124. It is used by the `date_format`, `from_unixtime`, `unix_timestamp` and `to_unix_timestamp` functions.

The PR aims to make Spark 3.0 compatible with Spark 2.4.x in all those cases when `spark.sql.legacy.timeParser.enabled` is set to `true`.

### Does this PR introduce any user-facing change?
This shouldn't change behavior with default settings. If `spark.sql.legacy.timeParser.enabled` is set to `true`, users should observe behavior of Spark 2.4.

### How was this patch tested?
- Modified tests in `DateExpressionsSuite` to check the legacy parser - `SimpleDateFormat`.
- Added `CSVLegacyTimeParserSuite` and `JsonLegacyTimeParserSuite` to run `CSVSuite` and `JsonSuite` with the legacy parser - `FastDateFormat`.

Closes #27524 from MaxGekk/timestamp-formatter-legacy-fallback.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 20:12:38 +08:00
turbofei 8b1839728a [SPARK-29542][FOLLOW-UP] Keep the description of spark.sql.files.* in tuning guide be consistent with that in SQLConf
### What changes were proposed in this pull request?
This pr is a follow up of https://github.com/apache/spark/pull/26200.

In this PR, I modify the description of spark.sql.files.* in sql-performance-tuning.md to keep consistent with that in SQLConf.

### Why are the changes needed?

To keep consistent with the description in SQLConf.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existed UT.

Closes #27545 from turboFei/SPARK-29542-follow-up.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 20:21:52 +09:00
beliefer f5026b1ba7 [SPARK-30763][SQL] Fix java.lang.IndexOutOfBoundsException No group 1 for regexp_extract
### What changes were proposed in this pull request?
The current implement of `regexp_extract` will throws a unprocessed exception show below:

`SELECT regexp_extract('1a 2b 14m', 'd+')`
```
java.lang.IndexOutOfBoundsException: No group 1
[info] at java.util.regex.Matcher.group(Matcher.java:538)
[info] at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
[info] at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
[info] at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
```
I think should treat this exception well.

### Why are the changes needed?
Fix a bug `java.lang.IndexOutOfBoundsException No group 1 `

### Does this PR introduce any user-facing change?
Yes

### How was this patch tested?
New UT

Closes #27508 from beliefer/fix-regexp_extract-bug.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 14:49:22 +08:00
Kris Mok b4769998ef [SPARK-30795][SQL] Spark SQL codegen's code() interpolator should treat escapes like Scala's StringContext.s()
### What changes were proposed in this pull request?

This PR proposes to make the `code` string interpolator treat escapes the same way as Scala's builtin `StringContext.s()` string interpolator. This will remove the need for an ugly workaround in `Like` expression's codegen.

### Why are the changes needed?

The `code()` string interpolator in Spark SQL's code generator should treat escapes like Scala's builtin `StringContext.s()` interpolator, i.e. it should treat escapes in the code parts, and should not treat escapes in the input arguments.

For example,
```scala
val arg = "This is an argument."
val str = s"This is string part 1. $arg This is string part 2."
val code = code"This is string part 1. $arg This is string part 2."
assert(code.toString == str)
```
We should expect the `code()` interpolator to produce the same result as the `StringContext.s()` interpolator, where only escapes in the string parts should be treated, while the args should be kept verbatim.

But in the current implementation, due to the eager folding of code parts and literal input args, the escape treatment is incorrectly done on both code parts and literal args.
That causes a problem when an arg contains escape sequences and wants to preserve that in the final produced code string. For example, in `Like` expression's codegen, there's an ugly workaround for this bug:
```scala
      // We need double escape to avoid org.codehaus.commons.compiler.CompileException.
      // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'.
      // '\"' will cause exception 'Line break in literal not allowed'.
      val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') {
        s"""\\\\\\$escapeChar"""
      } else {
        escapeChar
      }
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Added a new unit test case in `CodeBlockSuite`.

Closes #27544 from rednaxelafx/fix-code-string-interpolator.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 15:19:16 +09:00
HyukjinKwon aa6a60530e [SPARK-30722][PYTHON][DOCS] Update documentation for Pandas UDF with Python type hints
### What changes were proposed in this pull request?

This PR targets to document the Pandas UDF redesign with type hints introduced at SPARK-28264.
Mostly self-describing; however, there are few things to note for reviewers.

1. This PR replace the existing documentation of pandas UDFs to the newer redesign to promote the Python type hints. I added some words that Spark 3.0 still keeps the compatibility though.

2. This PR proposes to name non-pandas UDFs as "Pandas Function API"

3. SCALAR_ITER become two separate sections to reduce confusion:
  - `Iterator[pd.Series]` -> `Iterator[pd.Series]`
  - `Iterator[Tuple[pd.Series, ...]]` -> `Iterator[pd.Series]`

4. I removed some examples that look overkill to me.

5. I also removed some information in the doc, that seems duplicating or too much.

### Why are the changes needed?

To document new redesign in pandas UDF.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests should cover.

Closes #27466 from HyukjinKwon/SPARK-30722.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 10:49:46 +09:00
herman b25359cca3 [SPARK-30780][SQL] Empty LocalTableScan should use RDD without partitions
### What changes were proposed in this pull request?
This is a small follow-up for https://github.com/apache/spark/pull/27400. This PR makes an empty `LocalTableScanExec` return an `RDD` without partitions.

### Why are the changes needed?
It is a bit unexpected that the RDD contains partitions if there is not work to do. It also can save a bit of work when this is used in a more complex plan.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Added test to `SparkPlanSuite`.

Closes #27530 from hvanhovell/SPARK-30780.

Authored-by: herman <herman@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 10:48:29 +09:00
Maxim Gekk 45db48e2d2 Revert "[SPARK-30625][SQL] Support escape as third parameter of the like function
### What changes were proposed in this pull request?

In the PR, I propose to revert the commit 8aebc80e0e.

### Why are the changes needed?
See the concerns https://github.com/apache/spark/pull/27355#issuecomment-584344438

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By existing test suites.

Closes #27531 from MaxGekk/revert-like-3-args.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-11 10:15:34 -08:00
HyukjinKwon 99bd59fe29 [SPARK-29462][SQL][DOCS] Add some more context and details in 'spark.sql.defaultUrlStreamHandlerFactory.enabled' documentation
### What changes were proposed in this pull request?

This PR adds some more information and context to `spark.sql.defaultUrlStreamHandlerFactory.enabled`.

### Why are the changes needed?

It is a bit difficult to understand the documentation of `spark.sql.defaultUrlStreamHandlerFactory.enabled`.

### Does this PR introduce any user-facing change?

Nope, internal doc only fix.

### How was this patch tested?

Nope. I only tested linter.

Closes #27541 from HyukjinKwon/SPARK-29462-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-11 09:55:02 -08:00
Yin Huai ea626b6acf [SPARK-30783] Exclude hive-service-rpc
### What changes were proposed in this pull request?
Exclude hive-service-rpc from build.

### Why are the changes needed?
hive-service-rpc 2.3.6 and spark sql's thrift server module have duplicate classes. Leaving hive-service-rpc 2.3.6 in the class path means that spark can pick up classes defined in hive instead of its thrift server module, which can cause hard to debug runtime errors due to class loading order and compilation errors for applications depend on spark.

 If you compare hive-service-rpc 2.3.6's jar (https://search.maven.org/remotecontent?filepath=org/apache/hive/hive-service-rpc/2.3.6/hive-service-rpc-2.3.6.jar) and spark thrift server's jar (e.g. https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-hive-thriftserver_2.12/3.0.0-SNAPSHOT/spark-hive-thriftserver_2.12-3.0.0-20200207.021914-364.jar), you will see that all of classes provided by hive-service-rpc-2.3.6.jar are covered by spark thrift server's jar. https://issues.apache.org/jira/browse/SPARK-30783 has output of jar tf for both jars.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Existing tests.

Closes #27533 from yhuai/SPARK-30783.

Authored-by: Yin Huai <yhuai@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 00:12:45 +08:00
Maxim Gekk dc66d57e98 [SPARK-30754][SQL] Reuse results of floorDiv in calculations of floorMod in DateTimeUtils
### What changes were proposed in this pull request?
In the case of back-to-back calculation of `floorDiv` and `floorMod` with the same arguments, the result of `foorDiv` can be reused in calculation of `floorMod`. The `floorMod` method is defined as the following in Java standard library:
```java
    public static int floorMod(int x, int y) {
        int r = x - floorDiv(x, y) * y;
        return r;
    }
```
If `floorDiv(x, y)` has been already calculated, it can be reused in `x - floorDiv(x, y) * y`.

I propose to modify 2 places in `DateTimeUtils`:
1. `microsToInstant` which is widely used in many date-time functions. `Math.floorMod(us, MICROS_PER_SECOND)` is just replaced by its definition from Java Math library.
2. `truncDate`: `Math.floorMod(oldYear, divider) == 0` is replaced by `Math.floorDiv(oldYear, divider) * divider == oldYear` where `floorDiv(...) * divider` is pre-calculated.

### Why are the changes needed?
This reduces the number of arithmetic operations, and can slightly improve performance of date-time functions.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`.

Closes #27491 from MaxGekk/opt-microsToInstant.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-02-11 09:07:40 -06:00
fuwhu f1d0dce484 [MINOR][DOC] Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions
### What changes were proposed in this pull request?
Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions.

### Why are the changes needed?
To describe these two classes.

### Does this PR introduce any user-facing change?
no

### How was this patch tested?
no

Closes #27535 from fuwhu/SPARK-15616-FOLLOW-UP.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-11 22:16:44 +08:00
root1 b20754d9ee [SPARK-27545][SQL][DOC] Update the Documentation for CACHE TABLE and UNCACHE TABLE
### What changes were proposed in this pull request?
Document updated for `CACHE TABLE` & `UNCACHE TABLE`

### Why are the changes needed?
Cache table creates a temp view while caching data using `CACHE TABLE name AS query`. `UNCACHE TABLE` does not remove this temp view.

These things were not mentioned in the existing doc for `CACHE TABLE` & `UNCACHE TABLE`.

### Does this PR introduce any user-facing change?
Document updated for `CACHE TABLE` & `UNCACHE TABLE` command.

### How was this patch tested?
Manually

Closes #27090 from iRakson/SPARK-27545.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-11 20:42:02 +08:00
HyukjinKwon 0045be766b [SPARK-29462][SQL] The data type of "array()" should be array<null>
### What changes were proposed in this pull request?

This brings https://github.com/apache/spark/pull/26324 back. It was reverted basically because, firstly Hive compatibility, and the lack of investigations in other DBMSes and ANSI.

- In case of PostgreSQL seems coercing NULL literal to TEXT type.
- Presto seems coercing `array() + array(1)` -> array of int.
- Hive seems  `array() + array(1)` -> array of strings

 Given that, the design choices have been differently made for some reasons. If we pick one of both, seems coercing to array of int makes much more sense.

Another investigation was made offline internally. Seems ANSI SQL 2011, section 6.5 "<contextually typed value specification>" states:

> If ES is specified, then let ET be the element type determined by the context in which ES appears. The declared type DT of ES is Case:
>
> a) If ES simply contains ARRAY, then ET ARRAY[0].
>
> b) If ES simply contains MULTISET, then ET MULTISET.
>
> ES is effectively replaced by CAST ( ES AS DT )

From reading other related context, doing it to `NullType`. Given the investigation made, choosing to `null` seems correct, and we have a reference Presto now. Therefore, this PR proposes to bring it back.

### Why are the changes needed?
When empty array is created, it should be declared as array<null>.

### Does this PR introduce any user-facing change?
Yes, `array()` creates `array<null>`. Now `array(1) + array()` can correctly create `array(1)` instead of `array("1")`.

### How was this patch tested?
Tested manually

Closes #27521 from HyukjinKwon/SPARK-29462.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 17:22:08 +09:00
HyukjinKwon 2bc765a831 [SPARK-30756][SQL] Fix ThriftServerWithSparkContextSuite on spark-branch-3.0-test-sbt-hadoop-2.7-hive-2.3
### What changes were proposed in this pull request?

This PR tries #26710 (comment) way to fix the test.

### Why are the changes needed?

To make the tests pass.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Jenkins will test first, and then `on spark-branch-3.0-test-sbt-hadoop-2.7-hive-2.3` will test it out.

Closes #27513 from HyukjinKwon/test-SPARK-30756.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
(cherry picked from commit 8efe367a4e)
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 15:50:16 +09:00
Bryan Cutler 07a9885f27 [SPARK-30777][PYTHON][TESTS] Fix test failures for Pandas >= 1.0.0
### What changes were proposed in this pull request?

Fix PySpark test failures for using Pandas >= 1.0.0.

### Why are the changes needed?

Pandas 1.0.0 has recently been released and has API changes that result in PySpark test failures, this PR fixes the broken tests.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Manually tested with Pandas 1.0.1 and PyArrow 0.16.0

Closes #27529 from BryanCutler/pandas-fix-tests-1.0-SPARK-30777.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 10:03:01 +09:00
Shixiong Zhu e2ebca733c [SPARK-30779][SS] Fix some API issues found when reviewing Structured Streaming API docs
### What changes were proposed in this pull request?

- Fix the scope of `Logging.initializeForcefully` so that it doesn't appear in subclasses' public methods. Right now, `sc.initializeForcefully(false, false)` is allowed to called.
- Don't show classes under `org.apache.spark.internal` package in API docs.
- Add missing `since` annotation.
- Fix the scope of `ArrowUtils` to remove it from the API docs.

### Why are the changes needed?

Avoid leaking APIs unintentionally in Spark 3.0.0.

### Does this PR introduce any user-facing change?

No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?

Manually generated the API docs and verified the above issues have been fixed.

Closes #27528 from zsxwing/audit-ss-apis.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-10 14:26:14 -08:00
Yuanjian Li a6b91d2bf7 [SPARK-30556][SQL][FOLLOWUP] Reset the status changed in SQLExecution.withThreadLocalCaptured
### What changes were proposed in this pull request?
Follow up for #27267, reset the status changed in SQLExecution.withThreadLocalCaptured.

### Why are the changes needed?
For code safety.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existing UT.

Closes #27516 from xuanyuanking/SPARK-30556-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-10 22:16:25 +01:00
Maxim Gekk 3c1c9b48fc [SPARK-30759][SQL] Initialize cache for foldable patterns in StringRegexExpression
### What changes were proposed in this pull request?
In the PR, I propose to fix `cache` initialization in `StringRegexExpression` by changing `case Literal(value: String, StringType)` to `case p: Expression if p.foldable`

### Why are the changes needed?
Actually, the case doesn't work at all because of:
1. Literals value has type `UTF8String`
2. It doesn't work for foldable expressions like in the example:
```sql
SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*';
```
<img width="649" alt="Screen Shot 2020-02-08 at 22 45 50" src="https://user-images.githubusercontent.com/1580697/74091681-0d4a2180-4acb-11ea-8a0d-7e8c65f4214e.png">

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By the `check outputs of expression examples` test from `SQLQuerySuite`.

Closes #27502 from MaxGekk/str-regexp-foldable-pattern.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-10 12:51:37 -08:00
HyukjinKwon 4439b29bd2 Revert "[SPARK-30245][SQL] Add cache for Like and RLike when pattern is not static"
### What changes were proposed in this pull request?

This reverts commit 8ce7962931. There's variable name conflicts with 8aebc80e0e (diff-39298b470865a4cbc67398a4ea11e767).

This can be cleanly ported back to branch-3.0.

### Why are the changes needed?
Performance investigation were not made enough and it's not clear if it really beneficial or now.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Jenkins tests.

Closes #27514 from HyukjinKwon/revert-cache-PR.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-10 10:56:43 -08:00
Liang-Chi Hsieh acfdb46a60 [SPARK-27946][SQL][FOLLOW-UP] Change doc and error message for SHOW CREATE TABLE
### What changes were proposed in this pull request?

This is a follow-up for #24938 to tweak error message and migration doc.

### Why are the changes needed?

Making user know workaround if SHOW CREATE TABLE doesn't work for some Hive tables.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing unit tests.

Closes #27505 from viirya/SPARK-27946-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2020-02-10 10:45:00 -08:00
Eric Wu b2011a295b [SPARK-30326][SQL] Raise exception if analyzer exceed max iterations
### What changes were proposed in this pull request?
Enhance RuleExecutor strategy to take different actions when exceeding max iterations. And raise exception if analyzer exceed max iterations.

### Why are the changes needed?
Currently, both analyzer and optimizer just log warning message if rule execution exceed max iterations. They should have different behavior. Analyzer should raise exception to indicates the plan is not fixed after max iterations, while optimizer just log warning to keep the current plan. This is more feasible after SPARK-30138 was introduced.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Add test in AnalysisSuite

Closes #26977 from Eric5553/EnhanceMaxIterations.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 23:41:39 +08:00
jiake 5a240603fd [SPARK-30719][SQL] Add unit test to verify the log warning print when intentionally skip AQE
### What changes were proposed in this pull request?

This is a follow up in [#27452](https://github.com/apache/spark/pull/27452).
Add a unit test to verify whether the log warning is print when intentionally skip AQE.

### Why are the changes needed?

Add unit test

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

adding unit test

Closes #27515 from JkSelf/aqeLoggingWarningTest.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 21:48:00 +08:00
Terry Kim 70e545a94d [SPARK-30757][SQL][DOC] Update the doc on TableCatalog.alterTable's behavior
### What changes were proposed in this pull request?

This PR updates the documentation on `TableCatalog.alterTable`s behavior on the order by which the requested changes are applied. It now explicitly mentions that the changes are applied in the order given.

### Why are the changes needed?

The current documentation on `TableCatalog.alterTable` doesn't mention which order the requested changes are applied. It will be useful to explicitly document this behavior so that the user can expect the behavior. For example, `REPLACE COLUMNS` needs to delete columns before adding new columns, and if the order is guaranteed by `alterTable`, it's much easier to work with the catalog API.

### Does this PR introduce any user-facing change?

Yes, document change.

### How was this patch tested?

Not added (doc changes).

Closes #27496 from imback82/catalog_table_alter_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 19:04:49 +08:00