Commit graph

27446 commits

Author SHA1 Message Date
Yuanjian Li 86b54f3321 [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
### What changes were proposed in this pull request?
Introduce UnsafeRow format validation for streaming state store.

### Why are the changes needed?
Currently, Structured Streaming directly puts the UnsafeRow into StateStore without any schema validation. It's a dangerous behavior when users reusing the checkpoint file during migration. Any changes or bug fix related to the aggregate function may cause random exceptions, even the wrong answer, e.g SPARK-28067.

### Does this PR introduce _any_ user-facing change?
Yes. If the underlying changes are detected when the checkpoint is reused during migration, the InvalidUnsafeRowException will be thrown.

### How was this patch tested?
UT added. Will also add integrated tests for more scenario in another PR separately.

Closes #28707 from xuanyuanking/SPARK-31894.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 05:56:50 +00:00
Max Gekk 17a5007fd8 [SPARK-30865][SQL][SS] Refactor DateTimeUtils
### What changes were proposed in this pull request?

1. Move TimeZoneUTC and TimeZoneGMT to DateTimeTestUtils
2. Remove TimeZoneGMT
3. Use ZoneId.systemDefault() instead of defaultTimeZone().toZoneId
4. Alias SQLDate & SQLTimestamp to internal types of DateType and TimestampType
5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()`
6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()`
7. Remove `julianCommonEraStart`, `timestampToString()`, `microsToEpochDays()`, `epochDaysToMicros()`, `instantToDays()` from `DateTimeUtils`.
8. Make splitDate() private.
9. Remove `def daysToMicros(days: Int): Long` and `def microsToDays(micros: Long): Int`.

### Why are the changes needed?

This simplifies the common code related to date-time operations, and should improve maintainability. In particular:

1. TimeZoneUTC and TimeZoneGMT are moved to DateTimeTestUtils because they are used only in tests
2. TimeZoneGMT can be removed because it is equal to TimeZoneUTC
3. After the PR #27494, Spark expressions and DateTimeUtils functions switched to ZoneId instead of TimeZone completely. `defaultTimeZone()` with `TimeZone` as return type is not needed anymore.
4. SQLDate and SQLTimestamp types can be explicitly aliased to internal types of DateType and and TimestampType instead of declaring this in a comment.
5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()`.
6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()`.

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

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

Closes #27617 from MaxGekk/move-time-zone-consts.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 05:41:09 +00:00
Yuanjian Li 8750363c8d [MINOR][DOCS] Emphasize the Streaming tab is for DStream API
### What changes were proposed in this pull request?
Emphasize the Streaming tab is for DStream API.

### Why are the changes needed?
Some users reported that it's a little confusing of the streaming tab and structured streaming tab.

### Does this PR introduce _any_ user-facing change?
Document change.

### How was this patch tested?
N/A

Closes #28854 from xuanyuanking/minor-doc.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-19 12:17:40 +09:00
James Yu ac98a9a07f [MINOR][DOCS] Update running-on-kubernetes.md
### What changes were proposed in this pull request?

Fix executor container name typo. `executor` should be `spark-kubernetes-executor`.

### Why are the changes needed?

The Executor pod container name the users actually get from their Kubernetes clusters is different from that described in the documentation.

For example, below is what a user get from an executor pod.
```
Containers:
  spark-kubernetes-executor:
    Container ID:  docker://aaaabbbbccccddddeeeeffff
    Image:         <imagename>
    Image ID:      docker-pullable://0000.dkr.ecr.us-east-0.amazonaws.com/spark
    Port:          7079/TCP
    Host Port:     0/TCP
    Args:
      executor
    State:          Running
      Started:      Thu, 28 May 2020 05:54:04 -0700
    Ready:          True
    Restart Count:  0
    Limits:
      memory:  16Gi
```

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

Document change.

### How was this patch tested?

N/A

Closes #28862 from yuj/patch-1.

Authored-by: James Yu <yuj@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-18 14:36:20 -07:00
Wenchen Fan 8a9ae01e74 [MINOR] update dev/create-release/known_translations
This is auto-updated by running script `translate-contributors.py`

Closes #28861 from cloud-fan/update.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-18 15:32:44 +00:00
Dilip Biswal e4f5036146 [SPARK-32020][SQL] Better error message when SPARK_HOME or spark.test.home is not set
### What changes were proposed in this pull request?
Better error message when SPARK_HOME or spark,test.home is not set.

### Why are the changes needed?
Currently the error message is not easily consumable as it prints  (see below) the real error after printing the current environment which is rather long.

**Old output**
`
 time.name" -> "Java(TM) SE Runtime Environment", "sun.boot.library.path" -> "/Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/lib",
 "java.vm.version" -> "25.221-b11",
 . . .
 . . .
 . . .
) did not contain key "SPARK_HOME" spark.test.home or SPARK_HOME is not set.
	at org.scalatest.Assertions.newAssertionFailedExceptio
`

**New output**
An exception or error caused a run to abort: spark.test.home or SPARK_HOME is not set.
org.scalatest.exceptions.TestFailedException: spark.test.home or SPARK_HOME is not set
### Does this PR introduce any user-facing change?
`
No.

### How was this patch tested?
Ran the tests in intellej  manually to see the new error.

Closes #28825 from dilipbiswal/minor-spark-31950-followup.

Authored-by: Dilip Biswal <dkbiswal@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-18 22:45:55 +09:00
DB Tsai 9b792518b2 [SPARK-31960][YARN][BUILD] Only populate Hadoop classpath for no-hadoop build
### What changes were proposed in this pull request?
If a Spark distribution has built-in hadoop runtime, Spark will not populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` when a job is submitted to Yarn. Users can override this behavior by setting `spark.yarn.populateHadoopClasspath` to `true`.

### Why are the changes needed?
Without this, Spark will populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` even Spark distribution has built-in hadoop. This results jar conflict and many unexpected behaviors in runtime.

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

### How was this patch tested?
Manually test with two builds, with-hadoop and no-hadoop builds.

Closes #28788 from dbtsai/yarn-classpath.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-06-18 06:08:40 +00:00
yi.wu 4badef38a5 [SPARK-32000][CORE][TESTS] Fix the flaky test for partially launched task in barrier-mode
### What changes were proposed in this pull request?

This PR changes the test to get an active executorId and set it as preferred location instead of setting a fixed preferred location.

### Why are the changes needed?

The test is flaky. After checking the [log](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124086/artifact/core/), I find the root cause is:

Two test cases from different test suites got submitted at the same time because of concurrent execution. In this particular case, the two test cases (from DistributedSuite and BarrierTaskContextSuite) both launch under local-cluster mode. The two applications are submitted at the SAME time so they have the same applications(app-20200615210132-0000). Thus, when the cluster of BarrierTaskContextSuite is launching executors, it failed to create the directory for the executor 0, because the path (/home/jenkins/workspace/work/app-app-20200615210132-0000/0) has been used by the cluster of DistributedSuite. Therefore, it has to launch executor 1 and 2 instead, that lead to non of the tasks can get preferred locality thus they got scheduled together and lead to the test failure.

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

No.

### How was this patch tested?

The test can not be reproduced locally. We can only know it's been fixed when it's no longer flaky on Jenkins.

Closes #28849 from Ngone51/fix-spark-32000.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-17 13:28:47 +00:00
Max Gekk 350aa859fe [SPARK-32006][SQL] Create date/timestamp formatters once before collect in hiveResultString()
### What changes were proposed in this pull request?
1. Add method `getTimeFormatters` to `HiveResult` which creates timestamp and date formatters.
2. Move creation of `dateFormatter` and `timestampFormatter` from the constructor of the `HiveResult` object to `HiveResult. hiveResultString()` via `getTimeFormatters`. This allows to resolve time zone ID from Spark's session time zone `spark.sql.session.timeZone` and create date/timestamp formatters only once before collecting `java.sql.Timestamp`/`java.sql.Date` values.
3. Create date/timestamp formatters once in SparkExecuteStatementOperation.

### Why are the changes needed?
To fix perf regression comparing to Spark 2.4

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

### How was this patch tested?
- By existing test suite `HiveResultSuite` and etc.
- Re-generate benchmarks results of `DateTimeBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28842 from MaxGekk/opt-toHiveString-oss-master.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-17 06:28:47 +00:00
Dongjoon Hyun 93bb70f3fe [SPARK-29148][CORE][FOLLOWUP] Fix warning message to show a correct executor id
### What changes were proposed in this pull request?

This aims to replace `executorIdsToBeRemoved` with `executorIdToBeRemoved`.

### Why are the changes needed?

Since a wrong variable is used currently, `ArrayBuffer()` is always displayed.
```
20/06/16 19:33:31 WARN ExecutorAllocationManager: Not removing executor ArrayBuffer() because the ResourceProfile was UNKNOWN!
```

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

No.

### How was this patch tested?

Manual.

Closes #28847 from dongjoon-hyun/SPARK-29148.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-16 21:05:29 -07:00
HyukjinKwon feeca63198 [SPARK-32011][PYTHON][CORE] Remove warnings about pin-thread modes and guide to use collectWithJobGroup
### What changes were proposed in this pull request?

This PR proposes to remove the warning about multi-thread in local properties, and change the guide to use `collectWithJobGroup` for multi-threads for now because:
- It is too noisy to users who don't use multiple threads - the number of this single thread case is arguably more prevailing.
- There was a critical issue found about pin-thread mode SPARK-32010, which will be fixed in Spark 3.1.
- To smoothly migrate, `RDD.collectWithJobGroup` was added, which will be deprecated in Spark 3.1 with SPARK-32010 fixed.

I will target to deprecate `RDD.collectWithJobGroup`, and make this pin-thread mode stable in Spark 3.1. In the future releases, I plan to make this mode as a default mode, and remove `RDD.collectWithJobGroup` away.

### Why are the changes needed?

To avoid guiding users a feature with a critical issue, and provide a proper workaround for now.

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

Yes, warning message and documentation.

### How was this patch tested?

Manually tested:

Before:

```
>>> spark.sparkContext.setLocalProperty("a", "b")
/.../spark/python/pyspark/util.py:141: UserWarning: Currently, 'setLocalProperty' (set to local
properties) with multiple threads does not properly work.
Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple
threads on PVM, which fails to isolate local properties for each thread on PVM.
To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However,
note that it cannot inherit the local properties from the parent thread although it isolates each
thread on PVM and JVM with its own local properties.
To work around this, you should manually copy and set the local properties from the parent thread
 to the child thread when you create another thread.
```

After:
```
>>> spark.sparkContext.setLocalProperty("a", "b")
```

Closes #28845 from HyukjinKwon/SPARK-32011.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-17 12:10:12 +09:00
Max Gekk afd8a8b964 [SPARK-31989][SQL] Generate JSON rebasing files w/ 30 minutes step
### What changes were proposed in this pull request?
1. Change the max step from 1 week to 30 minutes in the tests `RebaseDateTimeSuite`.`generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'`.
2. Parallelise JSON files generation in the function `generateRebaseJson` by using `ThreadUtils.parmap`.

### Why are the changes needed?
1. To prevent the bugs that are fixed by https://github.com/apache/spark/pull/28787 and https://github.com/apache/spark/pull/28816.
2. The parallelisation speeds up JSON file generation.

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

### How was this patch tested?
By generating the JSON file `julian-gregorian-rebase-micros.json`.

Closes #28827 from MaxGekk/rebase-30-min.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-17 12:07:36 +09:00
Gabor Somogyi eeb81200e2 [SPARK-31337][SQL] Support MS SQL Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.

In this PR I've added MS SQL support.

What this PR contains:
* Added `MSSQLConnectionProvider`
* Added `MSSQLConnectionProviderSuite`
* Changed MS SQL JDBC driver to use the latest (test scope only)
* Changed `MsSqlServerIntegrationSuite` docker image to use the latest
* Added a version comment to `MariaDBConnectionProvider` to increase trackability

### Why are the changes needed?
Missing JDBC kerberos support.

### Does this PR introduce _any_ user-facing change?
Yes, now user is able to connect to MS SQL using kerberos.

### How was this patch tested?
* Additional + existing unit tests
* Existing integration tests
* Test on cluster manually

Closes #28635 from gaborgsomogyi/SPARK-31337.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-06-16 18:22:12 -07:00
Liang-Chi Hsieh 7f6a8ab166 [SPARK-31777][ML][PYSPARK] Add user-specified fold column to CrossValidator
### What changes were proposed in this pull request?

This patch adds user-specified fold column support to `CrossValidator`. User can assign fold numbers to dataset instead of letting Spark do random splits.

### Why are the changes needed?

This gives `CrossValidator` users more flexibility in splitting folds.

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

Yes, a new `foldCol` param is added to `CrossValidator`. User can use it to specify custom fold splitting.

### How was this patch tested?

Added unit tests.

Closes #28704 from viirya/SPARK-31777.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2020-06-16 16:46:32 -07:00
Zhen Li 2ec9b86628 [SPARK-31929][WEBUI] Close leveldbiterator when leveldb.close
### What changes were proposed in this pull request?

Close LevelDBIterator when LevelDB.close() is called.

### Why are the changes needed?

This pull request would prevent JNI resources leaking from Level DB instance and its' iterators. In before implementation JNI resources from LevelDBIterator are cleaned by finalize() function. This behavior is also mentioned in comments of ["LevelDBIterator.java"](https://github.com/apache/spark/blob/master/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java) by squito . But if DB instance is already closed, then iterator's close method would be ignored. LevelDB's iterator would keep level db files opened (for the case table cache is filled up), till iterator.close() is called. Then these JNI resources (file handle) would be leaked.
This JNI resource leaking issue would cause the problem described in [SPARK-31929](https://issues.apache.org/jira/browse/SPARK-31929) on Windows: in spark history server, leaked file handle for level db files would trigger "IOException" when HistoryServerDiskManager try to remove them for releasing disk space.
![IOException](https://user-images.githubusercontent.com/10524738/84134659-7c388680-aa7b-11ea-807f-04dcfa7886a0.JPG)

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

No

### How was this patch tested?

Add unit test and manually tested it.

Closes #28769 from zhli1142015/close-leveldbiterator-when-leveldb.close.

Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-16 12:59:57 -05:00
Takeshi Yamamuro 8d577092ed [SPARK-31705][SQL][FOLLOWUP] Avoid the unnecessary CNF computation for full-outer joins
### What changes were proposed in this pull request?

To avoid the unnecessary CNF computation for full-outer joins, this PR fixes code for filtering out full-outer joins at the entrance of the rule.

### Why are the changes needed?

To mitigate optimizer overhead.

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

No.

### How was this patch tested?

Existing tests.

Closes #28810 from maropu/SPARK-31705.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-06-16 09:13:00 -07:00
Max Gekk 36435658b1 [SPARK-31710][SQL][FOLLOWUP] Replace CAST by TIMESTAMP_SECONDS in benchmarks
### What changes were proposed in this pull request?
Replace `CAST(... AS TIMESTAMP` by `TIMESTAMP_SECONDS` in the following benchmarks:
- ExtractBenchmark
- DateTimeBenchmark
- FilterPushdownBenchmark
- InExpressionBenchmark

### Why are the changes needed?
The benchmarks fail w/o the changes:
```
[info] Running benchmark: datetime +/- interval
[info]   Running case: date + interval(m)
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`id` AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommend using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 5;
[error] 'Project [(cast(cast(id#0L as timestamp) as date) + 1 months) AS (CAST(CAST(id AS TIMESTAMP) AS DATE) + INTERVAL '1 months')#2]
[error] +- Range (0, 10000000, step=1, splits=Some(1))
```

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

### How was this patch tested?
By running the affected benchmarks.

Closes #28843 from MaxGekk/GuoPhilipse-31710-fix-compatibility-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 14:07:03 +00:00
Max Gekk 6e9ff72195 [SPARK-31984][SQL] Make micros rebasing functions via local timestamps pure
### What changes were proposed in this pull request?
1. Set the given time zone as the first parameter of `RebaseDateTime`.`rebaseJulianToGregorianMicros()` and `rebaseGregorianToJulianMicros()` to Java 7 `GregorianCalendar`.
```scala
    val cal = new Calendar.Builder()
      // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems
      .setCalendarType("gregory")
    ...
      .setTimeZone(tz)
      .build()
```
This makes the instance of the calendar independent from the default JVM time zone.

2. Change type of the first parameter from `ZoneId` to `TimeZone`. This allows to avoid unnecessary conversion from `TimeZone` to `ZoneId`, for example in
```scala
  def rebaseJulianToGregorianMicros(micros: Long): Long = {
    ...
      if (rebaseRecord == null || micros < rebaseRecord.switches(0)) {
        rebaseJulianToGregorianMicros(timeZone.toZoneId, micros)
```
and back to `TimeZone` inside of `rebaseJulianToGregorianMicros(zoneId: ZoneId, ...)`

3. Modify tests in `RebaseDateTimeSuite`, and set the default JVM time zone only for functions that depend on it.

### Why are the changes needed?
1. Ignoring passed parameter and using a global variable is bad practice.
2. Dependency from the global state doesn't allow to run the functions in parallel otherwise there is non-zero probability that the functions may return wrong result if the default JVM is changed during their execution.
3. This open opportunity for parallelisation of JSON files generation `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json`. Currently, the tests `generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'` generate the JSON files by iterating over all time zones sequentially w/ step of 1 week. Due to the large step, we can miss some spikes in diffs between 2 calendars (Java 8 Gregorian and Java 7 hybrid calendars) as the PR https://github.com/apache/spark/pull/28787 fixed and https://github.com/apache/spark/pull/28816 should fix.

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

### How was this patch tested?
By running existing tests from `RebaseDateTimeSuite`.

Closes #28824 from MaxGekk/pure-micros-rebasing.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 12:56:27 +00:00
yangjie01 d24d27f1bc [SPARK-31997][SQL][TESTS] Drop test_udtf table when SingleSessionSuite test completed
### What changes were proposed in this pull request?
`SingleSessionSuite` not do `DROP TABLE IF EXISTS test_udtf` when test completed, then if we do mvn test `HiveThriftBinaryServerSuite`, the test case named `SPARK-11595 ADD JAR with input path having URL scheme` will FAILED because it want to re-create an exists table test_udtf.

### Why are the changes needed?
test suite shouldn't rely on their execution order

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

### How was this patch tested?
Manual test,mvn test SingleSessionSuite and HiveThriftBinaryServerSuite in order

Closes #28838 from LuciferYang/drop-test-table.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-16 19:20:44 +09:00
GuoPhilipse f0e6d0ec13 [SPARK-31710][SQL] Fail casting numeric to timestamp by default
## What changes were proposed in this pull request?
we fail casting from numeric to timestamp by default.

## Why are the changes needed?
casting from numeric to timestamp is not a  non-standard,meanwhile it may generate different result between spark and other systems,for example hive

## Does this PR introduce any user-facing change?
Yes,user cannot cast numeric to timestamp directly,user have to use the following function to achieve the same effect:TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS

## How was this patch tested?
unit test added

Closes #28593 from GuoPhilipse/31710-fix-compatibility.

Lead-authored-by: GuoPhilipse <guofei_ok@126.com>
Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 08:35:35 +00:00
Jungtaek Lim (HeartSaVioR) fe68e95a5a [SPARK-24634][SS][FOLLOWUP] Rename the variable from "numLateInputs" to "numRowsDroppedByWatermark"
### What changes were proposed in this pull request?

This PR renames the variable from "numLateInputs" to "numRowsDroppedByWatermark" so that it becomes self-explanation.

### Why are the changes needed?

This is originated from post-review, see https://github.com/apache/spark/pull/28607#discussion_r439853232

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

No, as SPARK-24634 is not introduced in any release yet.

### How was this patch tested?

Existing UTs.

Closes #28828 from HeartSaVioR/SPARK-24634-v3-followup.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-16 16:41:08 +09:00
Max Gekk e9145d41f3 [SPARK-31986][SQL] Fix Julian-Gregorian micros rebasing of overlapping local timestamps
### What changes were proposed in this pull request?
It fixes microseconds rebasing from the hybrid calendar (Julian + Gregorian) to Proleptic Gregorian calendar in the function `RebaseDateTime`.`rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long` in the case of local timestamp overlapping.

In the case of overlapping, we look ahead of 1 day to determinate which instant we should take - earlier or later zoned timestamp. If our current standard zone and DST offsets are equal to zone offset of the next date, we choose the later timestamp otherwise the earlier one. For example, the local timestamp **1945-11-18 01:30:00.0** can be mapped to two instants (microseconds since 1970-01-01 00:00:00Z): -761211000000000 or -761207400000000. If the first one is passed to `rebaseJulianToGregorianMicros()`, we take the earlier instant in Proleptic Gregorian calendar while rebasing **1945-11-18T01:30+09:00[Asia/Hong_Kong]** otherwise the later one **1945-11-18T01:30+08:00[Asia/Hong_Kong]**.

Note: The fix assumes that only one transition of standard or DST offsets can occur during a day.

### Why are the changes needed?
Current implementation of `rebaseJulianToGregorianMicros()` handles timestamps overlapping only during daylight saving time but overlapping can happen also during transition from one standard time zone to another one. For example in the case of `Asia/Hong_Kong`, the time zone switched from `Japan Standard Time` (UTC+9) to `Hong Kong Time` (UTC+8) on _Sunday, 18 November, 1945 01:59:59 AM_. The changes allow to handle the special case as well.

### Does this PR introduce _any_ user-facing change?
There is no behaviour change for timestamps of CE after 0001-01-01. The PR might affects timestamps of BCE for which the modified `rebaseJulianToGregorianMicros()` is called directly.

### How was this patch tested?

1. By existing tests in `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `TimestampFormatterSuite`.

2. Added new checks to `RebaseDateTimeSuite`.`SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945`:
```scala
      assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros)
      assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros)
```

3. Regenerated `julian-gregorian-rebase-micros.json` with the step of 30 minutes, and got the same JSON file. The JSON file isn't affected because previously it was generated with the step of 1 week. And the spike in diffs/switch points during 1 hour of timestamp overlapping wasn't detected.

Closes #28816 from MaxGekk/fix-overlap-julian-2-grep.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 06:00:05 +00:00
Dongjoon Hyun 75afd88904 Revert "[SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber"
This reverts commit a0187cd6b5.
2020-06-15 19:04:23 -07:00
gengjiaan 5e89fbe44e [SPARK-31824][CORE][TESTS] DAGSchedulerSuite: Improve and reuse completeShuffleMapStageSuccessfully
### What changes were proposed in this pull request?
`DAGSchedulerSuite `provides `completeShuffleMapStageSuccessfully `to make `ShuffleMapStage `successfully.
But many test case uses complete directly as follows:
`complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1))))`

We need to improve `completeShuffleMapStageSuccessfully `and reuse it.
`completeShuffleMapStageSuccessfully(0, 0, 1, Some(0), Seq("hostA"))`

### Why are the changes needed?
Improve and reuse completeShuffleMapStageSuccessfully

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

### How was this patch tested?
Jenkins test

Closes #28641 from beliefer/improve-and-reuse-method.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-06-15 14:15:54 -07:00
Prashant Sharma a7d0d353cd [SPARK-31994][K8S] Docker image should use https urls for only deb.debian.org mirrors
### What changes were proposed in this pull request?
At the moment, we switch to `https` urls for all the debian mirrors, but turns out some of the mirrors do not support. In this patch, we turn on https mode only for `deb.debian.org` mirror (as it supports SSL).

### Why are the changes needed?
It appears, that security.debian.org does not support https.
```
curl https://security.debian.org
curl: (35) LibreSSL SSL_connect: SSL_ERROR_SYSCALL in connection to security.debian.org:443
```

While building the image, it fails in the following way.
```
MacBook-Pro:spark prashantsharma$ bin/docker-image-tool.sh -r scrapcodes -t v3.1.0-1 build
Sending build context to Docker daemon  222.1MB
Step 1/18 : ARG java_image_tag=8-jre-slim
Step 2/18 : FROM openjdk:${java_image_tag}
 ---> 381b20190cf7
Step 3/18 : ARG spark_uid=185
 ---> Using cache
 ---> 65c06f86753c
Step 4/18 : RUN set -ex &&     sed -i 's/http:/https:/g' /etc/apt/sources.list &&     apt-get update &&     ln -s /lib /lib64 &&     apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps &&     mkdir -p /opt/spark &&     mkdir -p /opt/spark/examples &&     mkdir -p /opt/spark/work-dir &&     touch /opt/spark/RELEASE &&     rm /bin/sh &&     ln -sv /bin/bash /bin/sh &&     echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su &&     chgrp root /etc/passwd && chmod ug+rw /etc/passwd &&     rm -rf /var/cache/apt/*
 ---> Running in a3461dadd6eb
+ sed -i s/http:/https:/g /etc/apt/sources.list
+ apt-get update
Ign:1 https://security.debian.org/debian-security buster/updates InRelease
Err:2 https://security.debian.org/debian-security buster/updates Release
  Could not handshake: The TLS connection was non-properly terminated. [IP: 151.101.0.204 443]
Get:3 https://deb.debian.org/debian buster InRelease [121 kB]
Get:4 https://deb.debian.org/debian buster-updates InRelease [51.9 kB]
Get:5 https://deb.debian.org/debian buster/main amd64 Packages [7905 kB]
Get:6 https://deb.debian.org/debian buster-updates/main amd64 Packages [7868 B]
Reading package lists...
E: The repository 'https://security.debian.org/debian-security buster/updates Release' does not have a Release file.
The command '/bin/sh -c set -ex &&     sed -i 's/http:/https:/g' /etc/apt/sources.list &&     apt-get update &&     ln -s /lib /lib64 &&     apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps &&     mkdir -p /opt/spark &&     mkdir -p /opt/spark/examples &&     mkdir -p /opt/spark/work-dir &&     touch /opt/spark/RELEASE &&     rm /bin/sh &&     ln -sv /bin/bash /bin/sh &&     echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su &&     chgrp root /etc/passwd && chmod ug+rw /etc/passwd &&     rm -rf /var/cache/apt/*' returned a non-zero code: 100
Failed to build Spark JVM Docker image, please refer to Docker build output for details.
```

So, if we limit the `https` support to only deb.debian.org, does the trick.

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

### How was this patch tested?
Manually, by building an image and testing it by running spark shell against it locally using kubernetes.

Closes #28834 from ScrapCodes/spark-31994/debian_mirror_fix.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-15 11:26:03 -07:00
Takeshi Yamamuro 3698a14204 [SPARK-26905][SQL] Follow the SQL:2016 reserved keywords
### What changes were proposed in this pull request?

This PR intends to move keywords `ANTI`, `SEMI`, and `MINUS` from reserved to non-reserved.

### Why are the changes needed?

To comply with the ANSI/SQL standard.

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

No.

### How was this patch tested?

Added tests.

Closes #28807 from maropu/SPARK-26905-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-06-16 00:27:45 +09:00
Max Gekk eae1747b66 [SPARK-31959][SQL][TESTS][FOLLOWUP] Adopt the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" to outdated tzdb
### What changes were proposed in this pull request?
Old JDK can have outdated time zone database in which `Asia/Hong_Kong` doesn't have timestamp overlapping in 1946 at all. This PR changes the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" in `RebaseDateTimeSuite`, and makes it tolerant to the case.

### Why are the changes needed?
To fix the test failures on old JDK w/ outdated tzdb like on Jenkins machine `research-jenkins-worker-09`.

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

### How was this patch tested?
By running the test on old JDK

Closes #28832 from MaxGekk/HongKong-tz-1945-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-15 08:09:07 -07:00
Takeshi Yamamuro 7f7b4dd519 [SPARK-31990][SS] Use toSet.toSeq in Dataset.dropDuplicates
### What changes were proposed in this pull request?

This PR partially revert SPARK-31292 in order to provide a hot-fix for a bug in `Dataset.dropDuplicates`; we must preserve the input order of `colNames` for `groupCols` because the Streaming's state store depends on the `groupCols` order.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

Added tests in `DataFrameSuite`.

Closes #28830 from maropu/SPARK-31990.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-15 07:48:48 -07:00
Huaxin Gao f83cb3cbb3 [SPARK-31925][ML] Summary.totalIterations greater than maxIters
### What changes were proposed in this pull request?
In LogisticRegression and LinearRegression, if set maxIter=n, the model.summary.totalIterations returns  n+1 if the training procedure does not drop out. This is because we use ```objectiveHistory.length``` as totalIterations, but ```objectiveHistory``` contains init sate, thus ```objectiveHistory.length``` is 1 larger than number of training iterations.

### Why are the changes needed?
correctness

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

### How was this patch tested?
add new tests and also modify existing tests

Closes #28786 from huaxingao/summary_iter.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-15 08:49:03 -05:00
Max Gekk 9d95f1b010 [SPARK-31992][SQL] Benchmark the EXCEPTION rebase mode
### What changes were proposed in this pull request?
- Modify `DateTimeRebaseBenchmark` to benchmark the default date-time rebasing mode - `EXCEPTION` for saving/loading dates/timestamps from/to parquet files. The mode is benchmarked for modern timestamps after 1900-01-01 00:00:00Z and dates after 1582-10-15.
- Regenerate benchmark results in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

### Why are the changes needed?
The `EXCEPTION` rebasing mode is the default mode of the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.datetimeRebaseModeInWrite`. The changes are needed to improve benchmark coverage for default settings.

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

### How was this patch tested?
By running the benchmark and check results manually.

Closes #28829 from MaxGekk/benchmark-exception-mode.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-15 07:25:56 +00:00
Kent Yao a0187cd6b5 [SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber
### What changes were proposed in this pull request?

This PR brings 02f32cfae4 back which reverted by 4a25200cd7 because of maven test failure

diffs newly made:
1. add a missing log4j file to test resources
2. Call `SessionState.detachSession()` to clean the thread local one in `afterAll`.
3. Not use dedicated JVMs for sbt test runner too

### Why are the changes needed?

fix the maven test

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

no

### How was this patch tested?

add new tests

Closes #28797 from yaooqinn/SPARK-31926-NEW.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-15 06:10:24 +00:00
Liang-Chi Hsieh 8282bbf12d [SPARK-27633][SQL] Remove redundant aliases in NestedColumnAliasing
## What changes were proposed in this pull request?

In NestedColumnAliasing rule, we create aliases for nested field access in project list. We considered that top level parent field and nested fields under it were both accessed. In the case, we don't create the aliases because they are redundant.

There is another case, where a nested parent field and nested fields under it were both accessed, which we don't consider now. We don't need to create aliases in this case too.

## How was this patch tested?

Added test.

Closes #24525 from viirya/SPARK-27633.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-15 11:01:56 +09:00
iRakson f5f6eee304 [SPARK-31642][FOLLOWUP] Fix Sorting for duration column and make Status column sortable
### What changes were proposed in this pull request?
In #28485 pagination support for tables of Structured Streaming Tab was added.
It missed 2 things:
* For sorting duration column, `String` was used which sometimes gives wrong results(consider `"3 ms"` and `"12 ms"`). Now we first sort the duration column and then convert it to readable String
* Status column was not made sortable.

### Why are the changes needed?
To fix the wrong result for sorting and making Status column sortable.

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

### How was this patch tested?
After changes:
<img width="1677" alt="Screenshot 2020-06-08 at 2 18 48 PM" src="https://user-images.githubusercontent.com/15366835/84010992-153fa280-a993-11ea-9846-bf176f2ec5d7.png">

Closes #28752 from iRakson/ssTests.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-14 16:41:59 -05:00
yi.wu 54e702c0dd [SPARK-31970][CORE] Make MDC configuration step be consistent between setLocalProperty and log4j.properties
### What changes were proposed in this pull request?

This PR proposes to use "mdc.XXX" as the consistent key for both `sc.setLocalProperty` and `log4j.properties` when setting up configurations for MDC.
### Why are the changes needed?

It's weird that we use "mdc.XXX" as key to set MDC value via `sc.setLocalProperty` while we use "XXX" as key to set MDC pattern in log4j.properties. It could also bring extra burden to the user.

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

No, as MDC feature is added in version 3.1, which hasn't been released.

### How was this patch tested?

Tested manually.

Closes #28801 from Ngone51/consistent-mdc.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-14 14:26:11 -07:00
uncleGen 1e40bccf44 [SPARK-31593][SS] Remove unnecessary streaming query progress update
### What changes were proposed in this pull request?

Structured Streaming progress reporter will always report an `empty` progress when there is no new data. As design, we should provide progress updates every 10s (default) when there is no new data.

Before PR:

![20200428175008](https://user-images.githubusercontent.com/7402327/80474832-88a8ca00-897a-11ea-820b-d4be6127d2fe.jpg)
![20200428175037](https://user-images.githubusercontent.com/7402327/80474844-8ba3ba80-897a-11ea-873c-b7137bd4a804.jpg)
![20200428175102](https://user-images.githubusercontent.com/7402327/80474848-8e061480-897a-11ea-806e-28c6bbf1fe03.jpg)

After PR:

![image](https://user-images.githubusercontent.com/7402327/80475099-f35a0580-897a-11ea-8fb3-53f343df2c3f.png)

### Why are the changes needed?

Fixes a bug around incorrect progress report

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

Fixes a bug around incorrect progress report

### How was this patch tested?

existing ut and manual test

Closes #28391 from uncleGen/SPARK-31593.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:49:01 +09:00
Jungtaek Lim (HeartSaVioR) 84815d0550 [SPARK-24634][SS] Add a new metric regarding number of inputs later than watermark plus allowed delay
### What changes were proposed in this pull request?

Please refer https://issues.apache.org/jira/browse/SPARK-24634 to see rationalization of the issue.

This patch adds a new metric to count the number of inputs arrived later than watermark plus allowed delay. To make changes simpler, this patch doesn't count the exact number of input rows which are later than watermark plus allowed delay. Instead, this patch counts the inputs which are dropped in the logic of operator. The difference of twos are shown in streaming aggregation: to optimize the calculation, streaming aggregation "pre-aggregates" the input rows, and later checks the lateness against "pre-aggregated" inputs, hence the number might be reduced.

The new metric will be provided via two places:

1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab
2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent.

### Why are the changes needed?

Dropping late inputs means that end users might not get expected outputs. Even end users may indicate the fact and tolerate the result (as that's what allowed lateness is for), but they should be able to observe whether the current value of allowed lateness drops inputs or not so that they can adjust the value.

Also, whatever the chance they have multiple of stateful operators in a single query, if Spark drops late inputs "between" these operators, it becomes "correctness" issue. Spark should disallow such possibility, but given we already provided the flexibility, at least we should provide the way to observe the correctness issue and decide whether they should make correction of their query or not.

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

Yes. End users will be able to retrieve the information of late inputs via two ways:

1. SQL tab in Spark UI
2. Streaming Query Listener

### How was this patch tested?

New UTs added & existing UTs are modified to reflect the change.

And ran manual test reproducing SPARK-28094.

I've picked the specific case on "B outer C outer D" which is enough to represent the "intermediate late row" issue due to global watermark.

https://gist.github.com/jammann/b58bfbe0f4374b89ecea63c1e32c8f17

Spark logs warning message on the query which means SPARK-28074 is working correctly,

```
20/05/30 17:52:47 WARN UnsupportedOperationChecker: Detected pattern of possible 'correctness' issue due to global watermark. The query contains stateful operation which can emit rows older than the current watermark plus allowed late record delay, which are "late rows" in downstream stateful operations and these rows can be discarded. Please refer the programming guide doc for more details.;
Join LeftOuter, ((D_FK#28 = D_ID#87) AND (B_LAST_MOD#26-T30000ms = D_LAST_MOD#88-T30000ms))
:- Join LeftOuter, ((C_FK#27 = C_ID#58) AND (B_LAST_MOD#26-T30000ms = C_LAST_MOD#59-T30000ms))
:  :- EventTimeWatermark B_LAST_MOD#26: timestamp, 30 seconds
:  :  +- Project [v#23.B_ID AS B_ID#25, v#23.B_LAST_MOD AS B_LAST_MOD#26, v#23.C_FK AS C_FK#27, v#23.D_FK AS D_FK#28]
:  :     +- Project [from_json(StructField(B_ID,StringType,false), StructField(B_LAST_MOD,TimestampType,false), StructField(C_FK,StringType,true), StructField(D_FK,StringType,true), value#21, Some(UTC)) AS v#23]
:  :        +- Project [cast(value#8 as string) AS value#21]
:  :           +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3a7fd18c, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable396d2958, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61a, [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> B, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#0, value#1, topic#2, partition#3, offset#4L, timestamp#5, timestampType#6]
:  +- EventTimeWatermark C_LAST_MOD#59: timestamp, 30 seconds
:     +- Project [v#56.C_ID AS C_ID#58, v#56.C_LAST_MOD AS C_LAST_MOD#59]
:        +- Project [from_json(StructField(C_ID,StringType,false), StructField(C_LAST_MOD,TimestampType,false), value#54, Some(UTC)) AS v#56]
:           +- Project [cast(value#41 as string) AS value#54]
:              +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3f507373, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable7b6736a4, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61b, [key#40, value#41, topic#42, partition#43, offset#44L, timestamp#45, timestampType#46], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> C, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#33, value#34, topic#35, partition#36, offset#37L, timestamp#38, timestampType#39]
+- EventTimeWatermark D_LAST_MOD#88: timestamp, 30 seconds
   +- Project [v#85.D_ID AS D_ID#87, v#85.D_LAST_MOD AS D_LAST_MOD#88]
      +- Project [from_json(StructField(D_ID,StringType,false), StructField(D_LAST_MOD,TimestampType,false), value#83, Some(UTC)) AS v#85]
         +- Project [cast(value#70 as string) AS value#83]
            +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider2b90e779, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable36f8cd29, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee620, [key#69, value#70, topic#71, partition#72, offset#73L, timestamp#74, timestampType#75], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> D, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#62, value#63, topic#64, partition#65, offset#66L, timestamp#67, timestampType#68]
```

and we can find the late inputs from the batch 4 as follows:

![Screen Shot 2020-05-30 at 18 02 53](https://user-images.githubusercontent.com/1317309/83324401-058fd200-a2a0-11ea-8bf6-89cf777e9326.png)

which represents intermediate inputs are being lost, ended up with correctness issue.

Closes #28607 from HeartSaVioR/SPARK-24634-v3.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:37:38 +09:00
TJX2014 a4ea599b1b [SPARK-31968][SQL] Duplicate partition columns check when writing data
### What changes were proposed in this pull request?
A unit test is added
Partition duplicate check added in `org.apache.spark.sql.execution.datasources.PartitioningUtils#validatePartitionColumn`

### Why are the changes needed?
When people write data with duplicate partition column, it will cause a `org.apache.spark.sql.AnalysisException: Found duplicate column ...` in loading data from the  writted.

### Does this PR introduce _any_ user-facing change?
Yes.
It will prevent people from using duplicate partition columns to write data.
1. Before the PR:
It will look ok at `df.write.partitionBy("b", "b").csv("file:///tmp/output")`,
but get an exception when read:
`spark.read.csv("file:///tmp/output").show()`
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the partition schema: `b`;
2. After the PR:
`df.write.partitionBy("b", "b").csv("file:///tmp/output")` will trigger the exception:
org.apache.spark.sql.AnalysisException: Found duplicate column(s) b, b: `b`;

### How was this patch tested?
Unit test.

Closes #28814 from TJX2014/master-SPARK-31968.

Authored-by: TJX2014 <xiaoxingstack@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-13 22:21:35 -07:00
Kousuke Saruta c2e5012a0a [SPARK-31632][CORE][WEBUI][FOLLOWUP] Enrich the exception message when application summary is unavailable
### 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 PR enriches the exception message when application summary is not available.
#28444 covers the case when application information is not available but the case application summary is not available is not covered.

### 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.
-->
To complement #28444 .

### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
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 possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
Yes.
Before this change, we can get the following error message when we access to `/jobs` if application summary is not available.
<img width="707" alt="no-such-element-exception-error-message" src="https://user-images.githubusercontent.com/4736016/84562182-6aadf200-ad8d-11ea-8980-d63edde6fad6.png">

After this change, we can get the following error message. It's like #28444 does.
<img width="1349" alt="enriched-errorm-message" src="https://user-images.githubusercontent.com/4736016/84562189-85806680-ad8d-11ea-8346-4da2ec11df2b.png">

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

I checked with the following procedure.
1. Set breakpoint in the line of `kvstore.write(appSummary)` in `AppStatusListener#onStartApplicatin`. Only the thread reaching this line should be suspended.
2. Start spark-shell and wait few seconds.
3. Access to `/jobs`

Closes #28820 from sarutak/fix-no-such-element.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:17:16 +09:00
Kousuke Saruta 610acb2fe4 [SPARK-31644][BUILD][FOLLOWUP] Make Spark's guava version configurable from the command line for sbt
### What changes were proposed in this pull request?

This PR proposes to support guava version configurable from command line for sbt.

### Why are the changes needed?

#28455 added the configurability for Maven but not for sbt.
sbt is usually faster than Maven so it's useful for developers.

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

No.

### How was this patch tested?

I confirmed the guava version is changed with the following commands.
```
 $ build/sbt "inspect tree clean"  | grep guava
[info]       +-spark/*:dependencyOverrides = Set(com.google.guava:guava:14.0.1, xerces:xercesImpl:2.12.0, jline:jline:2.14.6, org.apache.avro:avro:1.8.2)
```
```
$ build/sbt -Dguava.version=25.0-jre "inspect tree clean"  | grep guava
[info]       +-spark/*:dependencyOverrides = Set(com.google.guava:guava:25.0-jre, xerces:xercesImpl:2.12.0, jline:jline:2.14.6, org.apache.avro:avro:1.8.2)
```

Closes #28822 from sarutak/guava-version-for-sbt.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-13 19:04:33 -07:00
Huaxin Gao 89c98a4c70 [SPARK-31944] Add instance weight support in LinearRegressionSummary
### What changes were proposed in this pull request?
Add instance weight support in LinearRegressionSummary

### Why are the changes needed?
LinearRegression and RegressionMetrics support instance weight. We should support instance weight in LinearRegressionSummary too.

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

### How was this patch tested?
add new test

Closes #28772 from huaxingao/lir_weight_summary.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-13 12:20:29 -05:00
Gengliang Wang f535004e14 [SPARK-31967][UI] Downgrade to vis.js 4.21.0 to fix Jobs UI loading time regression
### What changes were proposed in this pull request?

After #28192, the job list page becomes very slow.
For example, after the following operation, the UI loading can take >40 sec.
```
(1 to 1000).foreach(_ => sc.parallelize(1 to 10).collect)
```

This is caused by a  [performance issue of `vis-timeline`](https://github.com/visjs/vis-timeline/issues/379). The serious issue affects both branch-3.0 and branch-2.4

I tried a different version 4.21.0 from https://cdnjs.com/libraries/vis
The infinite drawing issue seems also fixed if the zoom is disabled as default.

### Why are the changes needed?

Fix the serious perf issue in web UI by falling back vis-timeline-graph2d to an ealier version.

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

Yes, fix the UI perf regression

### How was this patch tested?

Manual test

Closes #28806 from gengliangwang/downgradeVis.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-12 17:22:41 -07:00
HyukjinKwon a620a2a7e5 [SPARK-31977][SQL] Returns the plan directly from NestedColumnAliasing
### What changes were proposed in this pull request?

This proposes a minor refactoring to match `NestedColumnAliasing` to `GeneratorNestedColumnAliasing` so it returns the pruned plan directly.

```scala
    case p  NestedColumnAliasing(nestedFieldToAlias, attrToAliases) =>
      NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
```

vs

```scala
    case GeneratorNestedColumnAliasing(p) => p
```

### Why are the changes needed?

Just for readability.

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

No.

### How was this patch tested?

Existing tests should cover.

Closes #28812 from HyukjinKwon/SPARK-31977.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-06-13 07:26:37 +09:00
Takeshi Yamamuro 78d08a8c38 [SPARK-31950][SQL][TESTS] Extract SQL keywords from the SqlBase.g4 file
### What changes were proposed in this pull request?

This PR intends to extract SQL reserved/non-reserved keywords from the ANTLR grammar file (`SqlBase.g4`) directly.

This approach is based on the cloud-fan suggestion: https://github.com/apache/spark/pull/28779#issuecomment-642033217

### Why are the changes needed?

It is hard to maintain a full set of the keywords in `TableIdentifierParserSuite`, so it would be nice if we could extract them from the `SqlBase.g4` file directly.

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

No.

### How was this patch tested?

Existing tests.

Closes #28802 from maropu/SPARK-31950-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-06-13 07:12:27 +09:00
Wenchen Fan 28f131fc8a [SPARK-31979] Release script should not fail when remove non-existing files
### What changes were proposed in this pull request?

When removing non-existing files in the release script, do not fail.

### Why are the changes needed?

This is to make the release script more robust, as we don't care if the files exist before we remove them.

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

no

### How was this patch tested?

tested when cutting 3.0.0 RC

Closes #28815 from cloud-fan/release.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-12 11:06:52 -07:00
iRakson 9b098f1eb9 [SPARK-30119][WEBUI] Support pagination for streaming tab
### What changes were proposed in this pull request?
#28747 reverted #28439 due to some flaky test case. This PR fixes the flaky test and adds pagination support.

### Why are the changes needed?
To support pagination for streaming tab

### Does this PR introduce _any_ user-facing change?
Yes, Now streaming tab tables will be paginated.

### How was this patch tested?
Manually.

Closes #28748 from iRakson/fixstreamingpagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-12 10:27:31 -05:00
Wenchen Fan d3a5e2963c Revert "[SPARK-31860][BUILD] only push release tags on succes"
This reverts commit 69ba9b662e.
2020-06-12 17:50:43 +08:00
Liang-Chi Hsieh ff89b11143 [SPARK-31736][SQL] Nested column aliasing for RepartitionByExpression/Join
### What changes were proposed in this pull request?

Currently we only push nested column pruning through a few operators such as LIMIT, SAMPLE, etc. This patch extends the feature to other operators including RepartitionByExpression, Join.

### Why are the changes needed?

Currently nested column pruning only applied on a few operators. It limits the benefit of nested column pruning. Extending nested column pruning coverage to make this feature more generally applied through different queries.

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

Yes. More SQL operators are covered by nested column pruning.

### How was this patch tested?

Added unit test, end-to-end tests.

Closes #28556 from viirya/others-column-pruning.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-12 16:54:55 +09:00
Max Gekk c259844df8 [SPARK-31959][SQL][TEST-JAVA11] Fix Gregorian-Julian micros rebasing while switching standard time zone offset
### What changes were proposed in this pull request?
Fix the bug in microseconds rebasing during transitions from one standard time zone offset to another one. In the PR, I propose to change the implementation of `rebaseGregorianToJulianMicros` which performs rebasing via local timestamps. In the case of overlapping:
1. Check that the original instant belongs to earlier or later instant of overlapped local timestamp.
2. If it is an earlier instant, take zone and DST offsets from the previous day otherwise
3. Set time zone offsets to Julian timestamp from the next day.

Note: The fix assumes that transitions cannot happen more often than once per 2 days.

### Why are the changes needed?
Current implementation handles timestamps overlapping only during daylight saving time but overlapping can happen also during transition from one standard time zone to another one. For example in the case of `Asia/Hong_Kong`, the time zone switched from `Japan Standard Time` (UTC+9) to `Hong Kong Time` (UTC+8) on _Sunday, 18 November, 1945 01:59:59 AM_. The changes allow to handle the special case as well.

### Does this PR introduce _any_ user-facing change?
It might affect micros rebasing in before common era when not-optimised version of `rebaseGregorianToJulianMicros()` is used directly.

### How was this patch tested?
1. By existing tests in `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `TimestampFormatterSuite`.
2. Added new test to `RebaseDateTimeSuite`
3. Regenerated `gregorian-julian-rebase-micros.json` with the step of 30 minutes, and got the same JSON file. The JSON file isn't affected because previously it was generated with the step of 1 week. And the spike in diffs/switch points during 1 hour of timestamp overlapping wasn't detected.

Closes #28787 from MaxGekk/HongKong-tz-1945.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-12 06:17:31 +00:00
Yuming Wang 78f9043862 [SPARK-31912][SQL][TESTS] Normalize all binary comparison expressions
### What changes were proposed in this pull request?

This pr normalize all binary comparison expressions when comparing plans.

### Why are the changes needed?

Improve test framework, otherwise this test will fail:
```scala
  test("SPARK-31912 Normalize all binary comparison expressions") {
    val original = testRelation
      .where('a === 'b && Literal(13) >= 'b).as("x")
    val optimized = testRelation
      .where(IsNotNull('a) && IsNotNull('b) && 'a === 'b && 'b <= 13 && 'a <= 13).as("x")
    comparePlans(Optimize.execute(original.analyze), optimized.analyze)
  }
```

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

### How was this patch tested?

Manual test.

Closes #28734 from wangyum/SPARK-31912.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-06-11 22:50:36 -07:00
Dilip Biswal b87a342c7d [SPARK-31916][SQL] StringConcat can lead to StringIndexOutOfBoundsException
### What changes were proposed in this pull request?
A minor fix to fix the append method of StringConcat to cap the length at MAX_ROUNDED_ARRAY_LENGTH to make sure it does not overflow and cause StringIndexOutOfBoundsException

Thanks to **Jeffrey Stokes** for reporting the issue and explaining the underlying problem in detail in the JIRA.

### Why are the changes needed?
This fixes StringIndexOutOfBoundsException on an overflow.

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

### How was this patch tested?
Added a test in StringsUtilSuite.

Closes #28750 from dilipbiswal/SPARK-31916.

Authored-by: Dilip Biswal <dkbiswal@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-06-12 09:19:29 +09:00