Commit graph

291 commits

Author SHA1 Message Date
Kent Yao 9562b26914 [SPARK-29757][SQL] Move calendar interval constants together
### What changes were proposed in this pull request?
```java
  public static final int YEARS_PER_DECADE = 10;
  public static final int YEARS_PER_CENTURY = 100;
  public static final int YEARS_PER_MILLENNIUM = 1000;

  public static final byte MONTHS_PER_QUARTER = 3;
  public static final int MONTHS_PER_YEAR = 12;

  public static final byte DAYS_PER_WEEK = 7;
  public static final long DAYS_PER_MONTH = 30L;

  public static final long HOURS_PER_DAY = 24L;

  public static final long MINUTES_PER_HOUR = 60L;

  public static final long SECONDS_PER_MINUTE = 60L;
  public static final long SECONDS_PER_HOUR = MINUTES_PER_HOUR * SECONDS_PER_MINUTE;
  public static final long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR;

  public static final long MILLIS_PER_SECOND = 1000L;
  public static final long MILLIS_PER_MINUTE = SECONDS_PER_MINUTE * MILLIS_PER_SECOND;
  public static final long MILLIS_PER_HOUR = MINUTES_PER_HOUR * MILLIS_PER_MINUTE;
  public static final long MILLIS_PER_DAY = HOURS_PER_DAY * MILLIS_PER_HOUR;

  public static final long MICROS_PER_MILLIS = 1000L;
  public static final long MICROS_PER_SECOND = MILLIS_PER_SECOND * MICROS_PER_MILLIS;
  public static final long MICROS_PER_MINUTE = SECONDS_PER_MINUTE * MICROS_PER_SECOND;
  public static final long MICROS_PER_HOUR = MINUTES_PER_HOUR * MICROS_PER_MINUTE;
  public static final long MICROS_PER_DAY = HOURS_PER_DAY * MICROS_PER_HOUR;
  public static final long MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY;
  /* 365.25 days per year assumes leap year every four years */
  public static final long MICROS_PER_YEAR = (36525L * MICROS_PER_DAY) / 100;

  public static final long NANOS_PER_MICROS = 1000L;
  public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
  public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
```
The above parameters are defined in IntervalUtils, DateTimeUtils, and CalendarInterval, some of them are redundant, some of them are cross-referenced.

### Why are the changes needed?
To simplify code, enhance consistency and reduce risks

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

no
### How was this patch tested?

modified uts

Closes #26399 from yaooqinn/SPARK-29757.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 19:48:19 +08:00
Wenchen Fan 9b61f90987 [SPARK-29761][SQL] do not output leading 'interval' in CalendarInterval.toString
### What changes were proposed in this pull request?

remove the leading "interval" in `CalendarInterval.toString`.

### Why are the changes needed?

Although it's allowed to have "interval" prefix when casting string to int, it's not recommended.

This is also consistent with pgsql:
```
cloud0fan=# select interval '1' day;
 interval
----------
 1 day
(1 row)
```

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

yes, when display a dataframe with interval type column, the result is different.

### How was this patch tested?

updated tests.

Closes #26401 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 15:44:50 +08:00
Maxim Gekk 29dc59ac29 [SPARK-29605][SQL] Optimize string to interval casting
### What changes were proposed in this pull request?
In the PR, I propose new function `stringToInterval()` in `IntervalUtils` for converting `UTF8String` to `CalendarInterval`. The function is used in casting a `STRING` column to an `INTERVAL` column.

### Why are the changes needed?
The proposed implementation is ~10 times faster. For example, parsing 9 interval units on JDK 8:
Before:
```
9 units w/ interval                               14004          14125         116          0.1       14003.6       0.0X
9 units w/o interval                              13785          14056         290          0.1       13784.9       0.0X
```
After:
```
9 units w/ interval                                1343           1344           1          0.7        1343.0       0.3X
9 units w/o interval                               1345           1349           8          0.7        1344.6       0.3X
```

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

### How was this patch tested?
- By new tests for `stringToInterval` in `IntervalUtilsSuite`
- By existing tests

Closes #26256 from MaxGekk/string-to-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 12:39:52 +08:00
Maxim Gekk 441d4c953e [SPARK-29723][SQL] Get date and time parts of an interval as java classes
### What changes were proposed in this pull request?
I propose 2 new methods for `CalendarInterval`:
- `extractAsPeriod()` returns the date part of an interval as an instance of `java.time.Period`
- `extractAsDuration()` returns the time part of an interval as an instance of `java.time.Duration`

For example:
```scala
scala> import org.apache.spark.unsafe.types.CalendarInterval
scala> import java.time._
scala> val i = spark.sql("select interval 1 year 3 months 4 days 10 hours 30 seconds").collect()(0).getAs[CalendarInterval](0)
scala> LocalDate.of(2019, 11, 1).plus(i.period())
res8: java.time.LocalDate = 2021-02-05
scala> ZonedDateTime.parse("2019-11-01T12:13:14Z").plus(i.extractAsPeriod()).plus(i.extractAsDuration())
res9: java.time.ZonedDateTime = 2021-02-05T22:13:44Z
```

### Why are the changes needed?
Taking into account that `CalendarInterval` has been already partially exposed to users via the collect operation, and probably it will be fully exposed in the future, it could be convenient for users to get the date and time parts of intervals as java classes:
- to avoid unnecessary dependency from Spark's classes in user code
- to easily use external libraries that accept standard Java classes.

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

### How was this patch tested?
By new test in `CalendarIntervalSuite`.

Closes #26368 from MaxGekk/interval-java-period-duration.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-04 11:07:54 -08:00
Maxim Gekk fb60c2a170 [SPARK-29671][SQL] Simplify string representation of intervals
### What changes were proposed in this pull request?
In the PR, I propose to changed `CalendarInterval.toString`:
- to skip the `week` unit
- to convert `milliseconds` and `microseconds` as the fractional part of the `seconds` unit.

### Why are the changes needed?
To improve readability.

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

### How was this patch tested?
- By `CalendarIntervalSuite` and `IntervalUtilsSuite`
- `literals.sql`, `datetime.sql` and `interval.sql`

Closes #26367 from MaxGekk/interval-to-string-format.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 22:56:59 -08:00
Maxim Gekk 80a89873b2 [SPARK-29733][TESTS] Fix wrong order of parameters passed to assertEquals
### What changes were proposed in this pull request?
The `assertEquals` method of JUnit Assert requires the first parameter to be the expected value. In this PR, I propose to change the order of parameters when the expected value is passed as the second parameter.

### Why are the changes needed?
Wrong order of assert parameters confuses when the assert fails and the parameters have special string representation. For example:
```java
assertEquals(input1.add(input2), new CalendarInterval(5, 5, 367200000000L));
```
```
java.lang.AssertionError:
Expected :interval 5 months 5 days 101 hours
Actual   :interval 5 months 5 days 102 hours
```

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

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

Closes #26377 from MaxGekk/fix-order-in-assert-equals.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 11:21:28 -08:00
Liu,Linhong a4382f7fe1 [SPARK-29486][SQL] CalendarInterval should have 3 fields: months, days and microseconds
### What changes were proposed in this pull request?
Current CalendarInterval has 2 fields: months and microseconds. This PR try to change it
to 3 fields: months, days and microseconds. This is because one logical day interval may
have different number of microseconds (daylight saving).

### Why are the changes needed?
One logical day interval may have different number of microseconds (daylight saving).
For example, in PST timezone, there will be 25 hours from 2019-11-2 12:00:00 to
2019-11-3 12:00:00

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

### How was this patch tested?
unit test and new added test cases

Closes #26134 from LinhongLiu/calendarinterval.

Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-01 18:12:33 +08:00
Xingbo Jiang 8207c835b4 Revert "Prepare Spark release v3.0.0-preview-rc2"
This reverts commit 007c873ae3.
2019-10-30 17:45:44 -07:00
Xingbo Jiang 007c873ae3 Prepare Spark release v3.0.0-preview-rc2
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A
2019-10-30 17:42:59 -07:00
Maxim Gekk 44c1c03924 [SPARK-29607][SQL] Move static methods from CalendarInterval to IntervalUtils
### What changes were proposed in this pull request?
In the PR, I propose to move all static methods from the `CalendarInterval` class to the `IntervalUtils` object. All those methods are rewritten from Java to Scala.

### Why are the changes needed?
- For consistency with other helper methods. Such methods were placed to the helper object `IntervalUtils`, see https://github.com/apache/spark/pull/26190
- Taking into account that `CalendarInterval` will be fully exposed to users in the future (see https://github.com/apache/spark/pull/25022), it would be nice to clean it up by moving service methods to an internal object.

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

### How was this patch tested?
- By moved tests from `CalendarIntervalSuite` to `IntervalUtilsSuite`
- By existing test suites

Closes #26261 from MaxGekk/refactoring-calendar-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-30 01:15:18 +08:00
Xingbo Jiang b33a58c0c6 Revert "Prepare Spark release v3.0.0-preview-rc1"
This reverts commit 5eddbb5f1d.
2019-10-28 22:32:34 -07:00
Xingbo Jiang 5eddbb5f1d Prepare Spark release v3.0.0-preview-rc1
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A

Closes #26243 from jiangxb1987/3.0.0-preview-prepare.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-28 22:31:29 -07:00
Wenchen Fan cdea520ff8 [SPARK-29532][SQL] Simplify interval string parsing
### What changes were proposed in this pull request?

Only use antlr4 to parse the interval string, and remove the duplicated parsing logic from `CalendarInterval`.

### Why are the changes needed?

Simplify the code and fix inconsistent behaviors.

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

No

### How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #26190 from cloud-fan/parser.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-24 09:15:59 -07:00
Yuanjian Li 239ee3f561 [SPARK-9853][CORE] Optimize shuffle fetch of continuous partition IDs
This PR takes over #19788. After we split the shuffle fetch protocol from `OpenBlock` in #24565, this optimization can be extended in the new shuffle protocol. Credit to yucai, closes #19788.

### What changes were proposed in this pull request?
This PR adds the support for continuous shuffle block fetching in batch:

- Shuffle client changes:
    - Add new feature tag `spark.shuffle.fetchContinuousBlocksInBatch`, implement the decision logic in `BlockStoreShuffleReader`.
    - Merge the continuous shuffle block ids in batch if needed in ShuffleBlockFetcherIterator.
- Shuffle server changes:
    - Add support in `ExternalBlockHandler` for the external shuffle service side.
    - Make `ShuffleBlockResolver.getBlockData` accept getting block data by range.
- Protocol changes:
    - Add new block id type `ShuffleBlockBatchId` represent continuous shuffle block ids.
    - Extend `FetchShuffleBlocks` and `OneForOneBlockFetcher`.
    - After the new shuffle fetch protocol completed in #24565, the backward compatibility for external shuffle service can be controlled by `spark.shuffle.useOldFetchProtocol`.

### Why are the changes needed?
In adaptive execution, one reducer may fetch multiple continuous shuffle blocks from one map output file. However, as the original approach, each reducer needs to fetch those 10 reducer blocks one by one. This way needs many IO and impacts performance. This PR is to support fetching those continuous shuffle blocks in one IO (batch way). See below example:

The shuffle block is stored like below:
![image](https://user-images.githubusercontent.com/2989575/51654634-c37fbd80-1fd3-11e9-935e-5652863676c3.png)
The ShuffleId format is s"shuffle_$shuffleId_$mapId_$reduceId", referring to BlockId.scala.

In adaptive execution, one reducer may want to read output for reducer 5 to 14, whose block Ids are from shuffle_0_x_5 to shuffle_0_x_14.
Before this PR, Spark needs 10 disk IOs + 10 network IOs for each output file.
After this PR, Spark only needs 1 disk IO and 1 network IO. This way can reduce IO dramatically.

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

### How was this patch tested?
Add new UT.
Integrate test with setting `spark.sql.adaptive.enabled=true`.

Closes #26040 from xuanyuanking/SPARK-9853.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-17 14:47:56 +08:00
Liang-Chi Hsieh 93e71e60e6 [SPARK-29469][SHUFFLE] Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed
### What changes were proposed in this pull request?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE. This proposes to skip retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed.

### Why are the changes needed?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE:

```
2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while beginning fetch of 2 outstanding blocks (after 3 retries)
java.lang.NullPointerException
at org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
```

It was happened after BlockManager and ExternalBlockStoreClient was closed due to previous errors. In this cases, RetryingBlockFetcher does not need to retry. This NPE is harmless for job execution, but is a source of misleading when looking at log. Especially for end-users.

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

No

### How was this patch tested?

Existing tests.

Closes #26115 from viirya/SPARK-29469.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 13:11:07 +08:00
Maxim Gekk da576a737c [SPARK-29369][SQL] Support string intervals without the interval prefix
### What changes were proposed in this pull request?
In the PR, I propose to move interval parsing to `CalendarInterval.fromCaseInsensitiveString()` which throws an `IllegalArgumentException` for invalid strings, and reuse it from `CalendarInterval.fromString()`. The former one handles `IllegalArgumentException` only and returns `NULL` for invalid interval strings. This will allow to support interval strings without the `interval` prefix in casting strings to intervals and in interval type constructor because they use `fromString()` for parsing string intervals.

For example:
```sql
spark-sql> select cast('1 year 10 days' as interval);
interval 1 years 1 weeks 3 days
spark-sql> SELECT INTERVAL '1 YEAR 10 DAYS';
interval 1 years 1 weeks 3 days
```

### Why are the changes needed?
To maintain feature parity with PostgreSQL which supports interval strings without prefix:
```sql
# select interval '2 months 1 microsecond';
        interval
------------------------
 2 mons 00:00:00.000001
```
and to improve Spark SQL UX.

### Does this PR introduce any user-facing change?
Yes, previously parsing of interval strings without `interval` gives `NULL`:
```sql
spark-sql> select interval '2 months 1 microsecond';
NULL
```
After:
```sql
spark-sql> select interval '2 months 1 microsecond';
interval 2 months 1 microseconds
```

### How was this patch tested?
- Added new tests to `CalendarIntervalSuite.java`
- A test for casting strings to intervals in `CastSuite`
- Test for interval type constructor from strings in `ExpressionParserSuite`

Closes #26079 from MaxGekk/interval-str-without-prefix.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-14 23:34:18 +08:00
Maxim Gekk b10344956d [SPARK-29342][SQL] Make casting of string values to intervals case insensitive
### What changes were proposed in this pull request?

In the PR, I propose to pass the `Pattern.CASE_INSENSITIVE` flag while compiling interval patterns in `CalendarInterval`. This makes casting string values to intervals case insensitive and tolerant to case of the `interval`, `year(s)`, `month(s)`, `week(s)`, `day(s)`, `hour(s)`, `minute(s)`, `second(s)`, `millisecond(s)` and `microsecond(s)`.

### Why are the changes needed?
There are at least 2 reasons:
- To maintain feature parity with PostgreSQL which is not sensitive to case:
```sql
 # select cast('10 Days' as INTERVAL);
 interval
----------
 10 days
(1 row)
```
- Spark is tolerant to case of interval literals. Case insensitivity in casting should be convenient for Spark users.
```sql
spark-sql> SELECT INTERVAL 1 YEAR 1 WEEK;
interval 1 years 1 weeks
```

### Does this PR introduce any user-facing change?
Yes, current implementation produces `NULL` for `interval`, `year`, ... `microsecond` that are not in lower case.
Before:
```sql
spark-sql> SELECT CAST('INTERVAL 10 DAYS' as INTERVAL);
NULL
```
After:
```sql
spark-sql> SELECT CAST('INTERVAL 10 DAYS' as INTERVAL);
interval 1 weeks 3 days
```

### How was this patch tested?
- by new tests in `CalendarIntervalSuite.java`
- new test in `CastSuite`

Closes #26010 from MaxGekk/interval-case-insensitive.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-07 09:33:01 -07:00
Dongjoon Hyun bd031c2173 [SPARK-29307][BUILD][TESTS] Remove scalatest deprecation warnings
### What changes were proposed in this pull request?

This PR aims to remove `scalatest` deprecation warnings with the following changes.
- `org.scalatest.mockito.MockitoSugar` -> `org.scalatestplus.mockito.MockitoSugar`
- `org.scalatest.selenium.WebBrowser` -> `org.scalatestplus.selenium.WebBrowser`
- `org.scalatest.prop.Checkers` -> `org.scalatestplus.scalacheck.Checkers`
- `org.scalatest.prop.GeneratorDrivenPropertyChecks` -> `org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks`

### Why are the changes needed?

According to the Jenkins logs, there are 118 warnings about this.
```
 grep "is deprecated" ~/consoleText | grep scalatest | wc -l
     118
```

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

No.

### How was this patch tested?

After Jenkins passes, we need to check the Jenkins log.

Closes #25982 from dongjoon-hyun/SPARK-29307.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 21:00:11 -07:00
Yuanjian Li f725d472f5 [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files
After the newly added shuffle block fetching protocol in #24565, we can keep this work by extending the FetchShuffleBlocks message.

### What changes were proposed in this pull request?
In this patch, we achieve the indeterminate shuffle rerun by reusing the task attempt id(unique id within an application) in shuffle id, so that each shuffle write attempt has a different file name. For the indeterministic stage, when the stage resubmits, we'll clear all existing map status and rerun all partitions.

All changes are summarized as follows:
- Change the mapId to mapTaskAttemptId in shuffle related id.
- Record the mapTaskAttemptId in MapStatus.
- Still keep mapId in ShuffleFetcherIterator for fetch failed scenario.
- Add the determinate flag in Stage and use it in DAGScheduler and the cleaning work for the intermediate stage.

### Why are the changes needed?
This is a follow-up work for #22112's future improvment[1]: `Currently we can't rollback and rerun a shuffle map stage, and just fail.`

Spark will rerun a finished shuffle write stage while meeting fetch failures, currently, the rerun shuffle map stage will only resubmit the task for missing partitions and reuse the output of other partitions. This logic is fine in most scenarios, but for indeterministic operations(like repartition), multiple shuffle write attempts may write different data, only rerun the missing partition will lead a correctness bug. So for the shuffle map stage of indeterministic operations, we need to support rolling back the shuffle map stage and re-generate the shuffle files.

### Does this PR introduce any user-facing change?
Yes, after this PR, the indeterminate stage rerun will be accepted by rerunning the whole stage. The original behavior is aborting the stage and fail the job.

### How was this patch tested?
- UT: Add UT for all changing code and newly added function.
- Manual Test: Also providing a manual test to verify the effect.
```
import scala.sys.process._
import org.apache.spark.TaskContext

val determinateStage0 = sc.parallelize(0 until 1000 * 1000 * 100, 10)
val indeterminateStage1 = determinateStage0.repartition(200)
val indeterminateStage2 = indeterminateStage1.repartition(200)
val indeterminateStage3 = indeterminateStage2.repartition(100)
val indeterminateStage4 = indeterminateStage3.repartition(300)
val fetchFailIndeterminateStage4 = indeterminateStage4.map { x =>
if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId == 190 &&
  TaskContext.get.stageAttemptNumber == 0) {
  throw new Exception("pkill -f -n java".!!)
  }
  x
}
val indeterminateStage5 = fetchFailIndeterminateStage4.repartition(200)
val finalStage6 = indeterminateStage5.repartition(100).collect().distinct.length
```
It's a simple job with multi indeterminate stage, it will get a wrong answer while using old Spark version like 2.2/2.3, and will be killed after #22112. With this fix, the job can retry all indeterminate stage as below screenshot and get the right result.
![image](https://user-images.githubusercontent.com/4833765/63948434-3477de00-caab-11e9-9ed1-75abfe6d16bd.png)

Closes #25620 from xuanyuanking/SPARK-25341-8.27.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-23 16:16:52 +08:00
Dongjoon Hyun 76bc9db749 [SPARK-29191][TESTS][SQL] Add tag ExtendedSQLTest for SQLQueryTestSuite
### What changes were proposed in this pull request?

This PR aims to add tag `ExtendedSQLTest` for `SQLQueryTestSuite`.
This doesn't affect our Jenkins test coverage.
Instead, this tag gives us an ability to parallelize them by splitting this test suite and the other suites.

### Why are the changes needed?

`SQLQueryTestSuite` takes 45 mins alone because it has many SQL scripts to run.

<img width="906" alt="time" src="https://user-images.githubusercontent.com/9700541/65353553-4af0f100-dba2-11e9-9f2f-386742d28f92.png">

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

No.

### How was this patch tested?

```
build/sbt "sql/test-only *.SQLQueryTestSuite" -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest
...
[info] SQLQueryTestSuite:
[info] ScalaTest
[info] Run completed in 3 seconds, 147 milliseconds.
[info] Total number of tests run: 0
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0
[info] No tests were executed.
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[success] Total time: 22 s, completed Sep 20, 2019 12:23:13 PM
```

Closes #25872 from dongjoon-hyun/SPARK-29191.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-22 13:53:21 -07:00
colinma 076186e881 [SPARK-19147][CORE] Gracefully handle error in task after executor is stopped
### What changes were proposed in this pull request?

TransportClientFactory.createClient() is called by task and TransportClientFactory.close() is called by executor.
When stop the executor, close() will set workerGroup = null, NPE will occur in createClient which generate many exception in log.
For exception occurs after close(), treated it as an expected Exception
and transform it to InterruptedException which can be processed by Executor.

### Why are the changes needed?

The change can reduce the exception stack trace in log file, and user won't be confused by these excepted exception.

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

N/A

### How was this patch tested?

New tests are added in TransportClientFactorySuite and ExecutorSuite

Closes #25759 from colinmjj/spark-19147.

Authored-by: colinma <colinma@tencent.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-21 07:31:39 -05:00
Sean Owen c5d8a51f3b [MINOR][BUILD] Fix about 15 misc build warnings
### What changes were proposed in this pull request?

This addresses about 15 miscellaneous warnings that appear in the current build.

### Why are the changes needed?

No functional changes, it just slightly reduces the amount of extra warning output.

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

No.

### How was this patch tested?

Existing tests, run manually.

Closes #25852 from srowen/BuildWarnings.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-19 11:37:42 -07:00
Dongjoon Hyun 471a3eff51 [SPARK-28932][BUILD][FOLLOWUP] Switch to scala-library compile dependency for JDK11
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/25638 to switch `scala-library` from `test` dependency to `compile` dependency in `network-common` module.

### Why are the changes needed?

Previously, we added `scala-library` as a test dependency to resolve the followings, but it was insufficient to resolve. This PR aims to switch it to compile dependency.
```
$ java -version
openjdk version "11.0.3" 2019-04-16
OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.3+7)
OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.3+7, mixed mode)

$ mvn clean install -pl common/network-common -DskipTests
...
[INFO] --- scala-maven-plugin:4.2.0:doc-jar (attach-scaladocs)  spark-network-common_2.12 ---
error: fatal error: object scala in compiler mirror not found.
one error found
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
```

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

No.

### How was this patch tested?

Manually, run the following on JDK11.
```
$ mvn clean install -pl common/network-common -DskipTests
```

Closes #25800 from dongjoon-hyun/SPARK-28932-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-16 00:13:07 -07:00
Dongjoon Hyun 1f96ce5443 [SPARK-28932][BUILD] Add scala-library test dependency to network-common module for JDK11
### What changes were proposed in this pull request?

This PR adds `scala-library` test dependency to `network-common` module for JDK11.

### Why are the changes needed?

In JDK11, the following command fails due to scala library.
```
mvn clean install -pl common/network-common -DskipTests
```

**BEFORE**
```
...
error: fatal error: object scala in compiler mirror not found.
one error found
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
```

**AFTER**
```
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
```

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

No.

### How was this patch tested?

Manual. On JDK11, do the following.
```
mvn clean install -pl common/network-common -DskipTests
```

Closes #25638 from dongjoon-hyun/SPARK-28932.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-31 10:59:20 -07:00
younggyu chun 8535df7261 [MINOR] Fix typos in comments and replace an explicit type with <>
## What changes were proposed in this pull request?
This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+.

## How was this patch tested?
Manually tested.

Closes #25338 from younggyuchun/younggyu.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:47:11 -05:00
Jungtaek Lim (HeartSaVioR) 128ea37bda [SPARK-28601][CORE][SQL] Use StandardCharsets.UTF_8 instead of "UTF-8" string representation, and get rid of UnsupportedEncodingException
## What changes were proposed in this pull request?

This patch tries to keep consistency whenever UTF-8 charset is needed, as using `StandardCharsets.UTF_8` instead of using "UTF-8". If the String type is needed, `StandardCharsets.UTF_8.name()` is used.

This change also brings the benefit of getting rid of `UnsupportedEncodingException`, as we're providing `Charset` instead of `String` whenever possible.

This also changes some private Catalyst helper methods to operate on encodings as `Charset` objects rather than strings.

## How was this patch tested?

Existing unit tests.

Closes #25335 from HeartSaVioR/SPARK-28601.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-05 20:45:54 -07:00
Yuanjian Li db39f45baf [SPARK-28593][CORE] Rename ShuffleClient to BlockStoreClient which more close to its usage
## What changes were proposed in this pull request?

After SPARK-27677, the shuffle client not only handles the shuffle block but also responsible for local persist RDD blocks. For better code scalability and precise semantics(as the [discussion](https://github.com/apache/spark/pull/24892#discussion_r300173331)), here we did several changes:

- Rename ShuffleClient to BlockStoreClient.
- Correspondingly rename the ExternalShuffleClient to ExternalBlockStoreClient, also change the server-side class from ExternalShuffleBlockHandler to ExternalBlockHandler.
- Move MesosExternalBlockStoreClient to Mesos package.

Note, we still keep the name of BlockTransferService, because the `Service` contains both client and server, also the name of BlockTransferService is not referencing shuffle client only.

## How was this patch tested?

Existing UT.

Closes #25327 from xuanyuanking/SPARK-28593.

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>
2019-08-05 14:54:45 +08:00
Sean Owen c09675779b [SPARK-28604][ML] Use log1p(x) over log(1+x) and expm1(x) over exp(x)-1 for accuracy
## What changes were proposed in this pull request?

Use `log1p(x)` over `log(1+x)` and `expm1(x)` over `exp(x)-1` for accuracy, where possible. This should improve accuracy a tiny bit in ML-related calculations, and shouldn't hurt in any event.

## How was this patch tested?

Existing tests.

Closes #25337 from srowen/SPARK-28604.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-04 17:04:01 -05:00
Zhu, Lipeng d26642dbbc [SPARK-28107][SQL] Support 'DAY TO (HOUR|MINUTE|SECOND)', 'HOUR TO (MINUTE|SECOND)' and 'MINUTE TO SECOND'
## What changes were proposed in this pull request?
The interval conversion behavior is same with the PostgreSQL.

https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/interval.sql#L180-L203

## How was this patch tested?
UT.

Closes #25000 from lipzhu/SPARK-28107.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 18:01:42 -07:00
Dongjoon Hyun a5ff9221fc [SPARK-28308][CORE] CalendarInterval sub-second part should be padded before parsing
## What changes were proposed in this pull request?

The sub-second part of the interval should be padded before parsing. Currently, Spark gives a correct value only when there is 9 digits below `.`.
```
spark-sql> select interval '0 0:0:0.123456789' day to second;
interval 123 milliseconds 456 microseconds

spark-sql> select interval '0 0:0:0.12345678' day to second;
interval 12 milliseconds 345 microseconds

spark-sql> select interval '0 0:0:0.1234' day to second;
interval 1 microseconds
```

## How was this patch tested?

Pass the Jenkins with the fixed test cases.

Closes #25079 from dongjoon-hyun/SPARK-28308.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 19:40:41 -07:00
Gabor Somogyi e11a55827e [SPARK-28261][CORE] Fix client reuse test
## What changes were proposed in this pull request?

There is the following code in [TransportClientFactory#createClient](https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java#L150)
```
    int clientIndex = rand.nextInt(numConnectionsPerPeer);
    TransportClient cachedClient = clientPool.clients[clientIndex];
```
which choose a client from its pool randomly. If we are unlucky we might not get the max number of connections out, but less than that.

To prove that I've tried out the following test:
```java
  Test
  public void testRandom() {
    Random rand = new Random();
    Set<Integer> clients = Collections.synchronizedSet(new HashSet<>());
    long iterCounter = 0;
    while (true) {
      iterCounter++;
      int maxConnections = 4;
      clients.clear();
      for (int i = 0; i < maxConnections * 10; i++) {
        int clientIndex = rand.nextInt(maxConnections);
        clients.add(clientIndex);
      }
      if (clients.size() != maxConnections) {
        System.err.println("Unexpected clients size (iterCounter=" + iterCounter + "): " + clients.size() + ", maxConnections: " + maxConnections);
      }
      if (iterCounter % 100000 == 0) {
        System.out.println("IterCounter: " + iterCounter);
      }
    }
  }
```

Result:
```
Unexpected clients size (iterCounter=22388): 3, maxConnections: 4
Unexpected clients size (iterCounter=36244): 3, maxConnections: 4
Unexpected clients size (iterCounter=85798): 3, maxConnections: 4
IterCounter: 100000
Unexpected clients size (iterCounter=97108): 3, maxConnections: 4
Unexpected clients size (iterCounter=119121): 3, maxConnections: 4
Unexpected clients size (iterCounter=129948): 3, maxConnections: 4
Unexpected clients size (iterCounter=173736): 3, maxConnections: 4
Unexpected clients size (iterCounter=178138): 3, maxConnections: 4
Unexpected clients size (iterCounter=195108): 3, maxConnections: 4
IterCounter: 200000
Unexpected clients size (iterCounter=209006): 3, maxConnections: 4
Unexpected clients size (iterCounter=217105): 3, maxConnections: 4
Unexpected clients size (iterCounter=222456): 3, maxConnections: 4
Unexpected clients size (iterCounter=226899): 3, maxConnections: 4
Unexpected clients size (iterCounter=229101): 3, maxConnections: 4
Unexpected clients size (iterCounter=253549): 3, maxConnections: 4
Unexpected clients size (iterCounter=277550): 3, maxConnections: 4
Unexpected clients size (iterCounter=289637): 3, maxConnections: 4
...
```

In this PR I've adapted the test code not to have this flakyness.

## How was this patch tested?

Additional (not committed test) + existing unit tests in a loop.

Closes #25075 from gaborgsomogyi/SPARK-28261.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 11:10:03 -07:00
LantaoJin 0e421000e0 [SPARK-28160][CORE] Fix a bug that callback function may hang when unchecked exception missed
## What changes were proposed in this pull request?

This is very like #23590 .

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the response is large but no enough memory is available. However, when this happens, `TransportClient.sendRpcSync` will just hang forever if the timeout set to unlimited.

This PR catches `Throwable` and uses the error to complete `SettableFuture`.

## How was this patch tested?

I tested in my IDE by setting the value of size to -1 to verify the result. Without this patch, it won't be finished until timeout (May hang forever if timeout set to MAX_INT), or the expected `IllegalArgumentException` will be caught.
```java
Override
      public void onSuccess(ByteBuffer response) {
        try {
          int size = response.remaining();
          ByteBuffer copy = ByteBuffer.allocate(size); // set size to -1 in runtime when debug
          copy.put(response);
          // flip "copy" to make it readable
          copy.flip();
          result.set(copy);
        } catch (Throwable t) {
          result.setException(t);
        }
      }
```

Closes #24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: lajin <lajin@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-30 15:14:41 -05:00
“attilapiros” b71c130fc6 [SPARK-27622][CORE] Avoiding the network when block manager fetches disk persisted RDD blocks from the same host
## What changes were proposed in this pull request?

Before this PR during fetching a disk persisted RDD block the network was always used to get the requested block content even when both the source and fetcher executor was running on the same host.

The idea to access another executor local disk files by directly reading the disk comes from the external shuffle service where the local dirs are stored for each executor (block manager).

To make this possible the following changes are done:
- `RegisterBlockManager` message is extended with the `localDirs` which is stored by the block manager master for each block manager as a new property of the `BlockManagerInfo`
- `GetLocationsAndStatus` is extended with the requester host
- `BlockLocationsAndStatus` (the reply for `GetLocationsAndStatus` message) is extended with the an option of local directories, which is filled with a local directories of a same host executor (if there is any, otherwise None is used). This is where the block content can be read from.

Shuffle blocks are out of scope of this PR: there will be a separate PR opened for that (for another Jira issue).

## How was this patch tested?

With a new unit test in `BlockManagerSuite`. See the the test prefixed by "SPARK-27622: avoid the network when block requested from same host".

Closes #24554 from attilapiros/SPARK-27622.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-25 07:35:44 -07:00
ketank-new 1a3858a769 [SPARK-26985][CORE] Fix "access only some column of the all of columns " for big endian architecture BUG
continuation to https://github.com/apache/spark/pull/24788

## What changes were proposed in this pull request?

Changes are related to BIG ENDIAN system
This changes are done to

identify s390x platform.
use byteorder to BIG_ENDIAN for big endian systems
changes for 2 are done in access functions putFloats() and putDouble()

## How was this patch tested?

Changes have been tested to build successfully on s390x as well x86 platform to make sure build is successful.

Closes #24861 from ketank-new/ketan_latest_v2.3.2.

Authored-by: ketank-new <ketan22584@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 08:24:10 -05:00
Sean Owen 9df7587eea [MINOR][CORE] Fix line too long in TransportClientFactory
## What changes were proposed in this pull request?

See https://github.com/apache/spark/pull/24702/files#r296765487 -- this just fixes a Java style error. I'm not clear why the PR build didn't catch it.

## How was this patch tested?

N/A

Closes #24951 from srowen/SPARK-27989.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 11:17:54 -05:00
Jose Luis Pedrosa 0671395966 [SPARK-27989][CORE] Added retries on the connection to the driver for k8s
Disabled negative dns caching for docker images

Improved logging on DNS resolution, convenient for slow k8s clusters

## What changes were proposed in this pull request?
Added retries when building the connection to the driver in K8s.
In some scenarios DNS reslution can take more than the timeout.
Also openjdk-8 by default has negative dns caching enabled, which means even retries may not help depending on the times.

## How was this patch tested?
This patch was tested agains an specific k8s cluster with slow response time in DNS to ensure it woks.

Closes #24702 from jlpedrosa/feature/kuberetries.

Authored-by: Jose Luis Pedrosa <jlpedrosa@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 09:25:43 -05:00
Josh Rosen fc65e0fe2c [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes
## What changes were proposed in this pull request?

This PR significantly improves the performance of `UTF8String.replace()` by performing direct replacement over UTF8 bytes instead of decoding those bytes into Java Strings.

In cases where the search string is not found (i.e. no replacements are performed, a case which I expect to be common) this new implementation performs no object allocation or memory copying.

My implementation is modeled after `commons-lang3`'s `StringUtils.replace()` method. As part of my implementation, I needed a StringBuilder / resizable buffer, so I moved `UTF8StringBuilder` from the `catalyst` package to `unsafe`.

## How was this patch tested?

Copied tests from `StringExpressionSuite` to `UTF8StringSuite` and added a couple of new cases.

To evaluate performance, I did some quick local benchmarking by running the following code in `spark-shell` (with Java 1.8.0_191):

```scala
import org.apache.spark.unsafe.types.UTF8String

def benchmark(text: String, search: String, replace: String) {
  val utf8Text = UTF8String.fromString(text)
  val utf8Search = UTF8String.fromString(search)
  val utf8Replace = UTF8String.fromString(replace)

  val start = System.currentTimeMillis
  var i = 0
  while (i < 1000 * 1000 * 100) {
    utf8Text.replace(utf8Search, utf8Replace)
    i += 1
  }
  val end = System.currentTimeMillis

  println(end - start)
}

benchmark("ABCDEFGH", "DEF", "ZZZZ")  // replacement occurs
benchmark("ABCDEFGH", "Z", "")  // no replacement occurs
```

On my laptop this took ~54 / ~40 seconds seconds before this patch's changes and ~6.5 / ~3.8 seconds afterwards.

Closes #24707 from JoshRosen/faster-string-replace.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-06-19 15:21:26 -07:00
Sean Owen 4576dfde19 [SPARK-28066][CORE] Optimize UTF8String.trim() for common case of no whitespace
## What changes were proposed in this pull request?

UTF8String.trim() allocates a new object even if the string has no whitespace, when it can just return itself. A simple check for this case makes the method about 3x faster in the common case.

## How was this patch tested?

Existing tests.

A rough benchmark of 90% strings without whitespace (at ends), and 10% that do have whitespace, suggests the average runtime goes from 20 ns to 6 ns.

Closes #24884 from srowen/SPARK-28066.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-17 08:49:11 -07:00
Zhu, Lipeng 5700c39c89 [SPARK-27578][SQL] Support INTERVAL ... HOUR TO SECOND syntax
## What changes were proposed in this pull request?

Currently, SparkSQL can support interval format like this.
```sql
SELECT INTERVAL '0 23:59:59.155' DAY TO SECOND
 ```

Like Presto/Teradata, this PR aims to support grammar like below.
```sql
SELECT INTERVAL '23:59:59.155' HOUR TO SECOND
```

Although we can add a new function for this pattern, we had better extend the existing code to handle a missing day case. So, the following is also supported.
```sql
SELECT INTERVAL '23:59:59.155' DAY TO SECOND
SELECT INTERVAL '1 23:59:59.155' HOUR TO SECOND
```
Currently Vertica/Teradata/Postgresql/SQL Server have fully support of below interval functions.
- interval ... year to month
- interval ... day to hour
- interval ... day to minute
- interval ... day to second
- interval ... hour to minute
- interval ... hour to second
- interval ... minute to second

https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Literals/interval-qualifier.htm
df1a699e5b/src/test/regress/sql/interval.sql (L180-L203)
https://docs.teradata.com/reader/S0Fw2AVH8ff3MDA0wDOHlQ/KdCtT3pYFo~_enc8~kGKVw
https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/interval-literals?view=sql-server-2017

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24472 from lipzhu/SPARK-27578.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-13 10:12:55 -07:00
Dongjoon Hyun e561e92765 [SPARK-27981][CORE] Remove Illegal reflective access warning for java.nio.Bits.unaligned() in JDK9+
## What changes were proposed in this pull request?

This PR aims to remove the following warnings for `java.nio.Bits.unaligned` at JDK9/10/11/12. Please note that there are more warnings which is beyond of this PR's scope. JDK9+ shows the first warning only if you don't give `--illegal-access=warn`.

**BEFORE (Among 5 warnings, there is `java.nio.Bits.unaligned` warning at the startup)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to method java.nio.Bits.unaligned()
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:01:19 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016882712).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 4 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-2.7` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:08:27 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560017311171).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 2 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-3.2` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
19/06/08 10:52:06 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016330287).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
...
```

## How was this patch tested?

Manual. Run Spark command like `spark-shell` with `--driver-java-options=--illegal-access=warn` option in JDK9/10/11/12 environment.

Closes #24825 from dongjoon-hyun/SPARK-27981.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 16:39:32 -07:00
gatorsmile 2e84181ec3 [SPARK-27773][FOLLOW-UP] Fix Checkstyle failure
## What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-lint/

```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[99] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[101] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[103] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[105] (sizes) LineLength: Line is longer than 100 characters (found 103).
```

## How was this patch tested?
N/A

Closes #24760 from gatorsmile/updateYarnShuffleServiceMetrics.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-31 09:30:17 -07:00
Steven Rand 568512cc82 [SPARK-27773][SHUFFLE] add metrics for number of exceptions caught in ExternalShuffleBlockHandler
## What changes were proposed in this pull request?

Add a metric for number of exceptions caught in the `ExternalShuffleBlockHandler`, the idea being that spikes in this metric over some time window (or more desirably, the lack thereof) can be used as an indicator of the health of an external shuffle service. (Where "health" refers to its ability to successfully respond to client requests.)

## How was this patch tested?

Deployed a build of this PR to a YARN cluster, and confirmed that the NodeManagers' JMX metrics include `numCaughtExceptions`.

Closes #24645 from sjrand/SPARK-27773.

Authored-by: Steven Rand <srand@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-30 13:57:15 -07:00
Marcelo Vanzin 09ed64d795 [SPARK-27868][CORE] Better default value and documentation for socket server backlog.
First, there is currently no public documentation for this setting. So it's hard
to even know that it could be a problem if your application starts failing with
weird shuffle errors.

Second, the javadoc attached to the code was incorrect; the default value just uses
the default value from the JRE, which is 50, instead of having an unbounded queue
as the comment implies.

So use a default that is a "rounded" version of the JRE default, and provide
documentation explaining that this value may need to be adjusted. Also added
a log message that was very helpful in debugging an issue caused by this
problem.

Closes #24732 from vanzin/SPARK-27868.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 14:56:36 -07:00
Shixiong Zhu 04f142db9c
[SPARK-20547][REPL] Throw RemoteClassLoadedError for transient errors in ExecutorClassLoader
## What changes were proposed in this pull request?

`ExecutorClassLoader`'s `findClass` may fail to fetch a class due to transient exceptions. For example, when a task is interrupted, if `ExecutorClassLoader` is fetching a class, you may see `InterruptedException` or `IOException` wrapped by `ClassNotFoundException`, even if this class can be loaded. Then the result of `findClass` will be cached by JVM, and later when the same class is being loaded in the same executor, it will just throw NoClassDefFoundError even if the class can be loaded.

I found JVM only caches `LinkageError` and `ClassNotFoundException`. Hence in this PR, I changed ExecutorClassLoader to throw `RemoteClassLoadedError` if we cannot get a response from driver.

## How was this patch tested?

New unit tests.

Closes #24683 from zsxwing/SPARK-20547-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-05-28 12:56:14 -07:00
Yuanjian Li 8949bc7a3c [SPARK-27665][CORE] Split fetch shuffle blocks protocol from OpenBlocks
## What changes were proposed in this pull request?

As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks protocol to describe the fetch request for shuffle blocks, and it causes the extension work for shuffle fetching like #19788 and #24110 very awkward.
In this PR, we split the fetch request for shuffle blocks from OpenBlocks which named FetchShuffleBlocks. It's a loose bind with ShuffleBlockId and can easily extend by adding new fields in this protocol.

## How was this patch tested?

Existing and new added UT.

Closes #24565 from xuanyuanking/SPARK-27665.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-27 22:19:31 +08:00
“attilapiros” 1e87694f2b [SPARK-27677][CORE] Disable by default fetching of disk persisted RDD blocks via external shuffle service
## What changes were proposed in this pull request?

In the PR the config `spark.shuffle.service.fetch.rdd.enabled` default is changed to **false** to avoid breaking any compatibility with older external shuffle service installations. As external shuffle service is deployed separately and disk persisted RDD block fetching had even introduced new network messages (`RemoveBlocks` and `BlocksRemoved`) and changed the behaviour of the already existing fetching: extended it for RDD blocks.

## How was this patch tested?

With existing unit tests.

Closes #24697 from attilapiros/minor-ext-shuffle-fetch-disabled.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-24 11:58:26 -07:00
“attilapiros” e9f3f62b2c [SPARK-27677][CORE] Serve local disk persisted blocks by the external service after releasing executor by dynamic allocation
# What changes were proposed in this pull request?

## Problem statement

An executor which has persisted blocks does not consider to be idle and this way ready to be released by dynamic allocation after the regular timeout `spark.dynamicAllocation.executorIdleTimeout` but there is separate configuration `spark.dynamicAllocation.cachedExecutorIdleTimeout` which defaults to `Integer.MAX_VALUE`. This is because releasing the executor also means losing the persisted blocks (as the metadata for individual blocks called `BlockInfo` are kept in memory) and when the RDD is referenced latter on this lost blocks will be recomputed.
On the other hand keeping the executors too long without any task to work on is also a waste of resources (as executors are reserved for the application by the resource manager).

## Solution

This PR focuses on the first part of SPARK-25888: it extends the external shuffle service with the capability to serve RDD blocks which are persisted on the local disk store by the executors. Moreover when this feature is enabled by setting the `spark.shuffle.service.fetch.rdd.enabled` config to true and a block is reported to be persisted on to disk the external shuffle service instance running on the same host as the executor is also registered (along with the reporting block manager) as a possible location for fetching it.

## Some implementation detail

Some explanation about the decisions made during the development:
- the location list to fetch a block was randomized but the groups (same host, same rack, others) order was kept. In this PR the order of groups are kept and external shuffle service added to the end of the each group.
- `BlockManagerInfo` is not introduced for external shuffle service but only a lightweight solution is taken. A hash map from `BlockId` to `BlockStatus` is introduced. A type alias would make the source more readable but I know it is discouraged. On the other hand a new class wrapping this hash map would introduce unnecessary indirection.
- when this feature is on the cleanup triggered during removing of executors (which is handled in `ExternalShuffleBlockResolver`) is modified to keep the disk persisted RDD blocks. This cleanup is triggered in standalone mode when the `spark.storage.cleanupFilesAfterExecutorExit` config is set.
- the unpersisting of an RDD is extended to use the external shuffle service for disk persisted RDD blocks when the original executor which created the blocks are already released. New block transport messages are introduced to support this: `RemoveBlocks` and `BlocksRemoved`.

# How was this patch tested?

## Unit tests

### ExternalShuffleServiceSuite

Here the complete use case is tested by the "SPARK-25888: using external shuffle service fetching disk persisted blocks" with a tiny difference: here the executor is killed manually, this way the test is a bit faster than waiting for the idle timeout.

### ExternalShuffleBlockHandlerSuite

Tests the fetching of the RDD blocks via the external shuffle service.

### BlockManagerInfoSuite

This a new suite. As the `BlockManagerInfo` behaviour depends very much on whether the external shuffle service enabled or not all the tests are executed with and without it.

### BlockManagerSuite

Tests the sorting of the block locations.

## Manually on YARN

Spark App was:

~~~scala
package com.mycompany

import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.storage.StorageLevel

object TestAppDiskOnlyLevel {
  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setAppName("test-app")

    println("Attila: START")
    val sc = new SparkContext(conf)
    val rdd = sc.parallelize(0 until 100, 10)
      .map { i =>
        println(s"Attila: calculate first rdd i=$i")
        Thread.sleep(1000)
        i
      }

    rdd.persist(StorageLevel.DISK_ONLY)
    rdd.count()

    println("Attila: First RDD is processed, waiting for 60 sec")

    Thread.sleep(60 * 1000)

    println("Attila: Num executors must be 0 as executorIdleTimeout is way over")

    val rdd2 = sc.parallelize(0 until 10, 1)
      .map(i => (i, 1))
      .persist(StorageLevel.DISK_ONLY)

    rdd2.count()

    println("Attila: Second RDD with one partition (only one executors must be alive)")

    // reduce runs as user code to detect the empty seq (empty blocks)
    println("Calling collect on the first RDD: " + rdd.collect().reduce(_ + _))

    println("Attila: STOP")
  }
}
~~~

I have submitted with the following configuration:

~~~bash
spark-submit --master yarn \
  --conf spark.dynamicAllocation.enabled=true \
  --conf spark.dynamicAllocation.executorIdleTimeout=30 \
  --conf spark.dynamicAllocation.cachedExecutorIdleTimeout=90 \
  --class com.mycompany.TestAppDiskOnlyLevel dyn_alloc_demo-core_2.11-0.1.0-SNAPSHOT-jar-with-dependencies.jar
~~~

Checked the result by filtering for the side effect of the task calculations:

~~~bash
[userserver ~]$ yarn logs -applicationId application_1556299359453_0001 | grep "Attila: calculate" | wc -l
WARNING: YARN_OPTS has been replaced by HADOOP_OPTS. Using value of YARN_OPTS.
19/04/26 10:31:59 INFO client.RMProxy: Connecting to ResourceManager at apiros-1.gce.company.com/172.31.115.165:8032
100
~~~

So it is only 100 task execution and not 200 (which would be the case for re-computation).

Moreover from the submit/launcher log we can see executors really stopped in between (see the new total is 0 before the last line):
~~~
[userserver ~]$ grep "Attila: Num executors must be 0" -B 2 spark-submit.log
19/04/26 10:24:27 INFO cluster.YarnScheduler: Executor 9 on apiros-3.gce.company.com killed by driver.
19/04/26 10:24:27 INFO spark.ExecutorAllocationManager: Existing executor 9 has been removed (new total is 0)
Attila: Num executors must be 0 as executorIdleTimeout is way over
~~~

[Full spark submit log](https://github.com/attilapiros/spark/files/3122465/spark-submit.log)

I have done a test also after changing the `DISK_ONLY` storage level to `MEMORY_ONLY` for the first RDD. After this change during the 60sec waiting no executor was removed.

Closes #24499 from attilapiros/SPARK-25888-final.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-23 13:15:39 -07:00
David Navas 9e73be38a5 [SPARK-27726][CORE] Fix performance of ElementTrackingStore deletes when using InMemoryStore under high loads
The details of the PR are explored in-depth in the sub-tasks of the umbrella jira SPARK-27726.
Briefly:
  1. Stop issuing asynchronous requests to cleanup elements in the tracking store when a request is already pending
  2. Fix a couple of thread-safety issues (mutable state and mis-ordered updates)
  3. Move Summary deletion outside of Stage deletion loop like Tasks already are
  4. Reimplement multi-delete in a removeAllKeys call which allows InMemoryStore to implement it in a performant manner.
  5. Some generic typing and exception handling cleanup

We see about five orders of magnitude improvement in the deletion code, which for us is the difference between a server that needs restarting daily, and one that is stable over weeks.

Unit tests for the fire-once asynchronous code and the removeAll calls in both LevelDB and InMemoryStore are supplied.  It was noted that the testing code for the LevelDB and InMemoryStore is highly repetitive, and should probably be merged, but we did not attempt that in this PR.

A version of this code was run in our production 2.3.3 and we were able to sustain higher throughput without going into GC overload (which was happening on a daily basis some weeks ago).

A version of this code was also put under a purpose-built Performance Suite of tests to verify performance under both types of Store implementations for both before and after code streams and for both total and partial delete cases (this code is not included in this PR).

Closes #24616 from davidnavas/PentaBugFix.

Authored-by: David Navas <davidn@clearstorydata.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-21 10:22:21 -07:00
Yuming Wang 93b5a2b686 [SPARK-27610][FOLLOW-UP][YARN] Remove duplicate declaration of plugin maven-antrun-plugin
## What changes were proposed in this pull request?

This pr removes duplicate declaration of plugin `org.apache.maven.plugins:maven-antrun-plugin`:
```
[WARNING] Some problems were encountered while building the effective model for org.apache.spark:spark-network-yarn_2.12🫙3.0.0-SNAPSHOT
[WARNING] 'build.plugins.plugin.(groupId:artifactId)' must be unique but found duplicate declaration of plugin org.apache.maven.plugins:maven-antrun-plugin  line 177, column 15
[WARNING]
[WARNING] It is highly recommended to fix these problems because they threaten the stability of your build.
[WARNING]
[WARNING] For this reason, future Maven versions might no longer support building such malformed projects.
[WARNING]
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105523/consoleFull

## How was this patch tested?

Existing test

Closes #24641 from wangyum/SPARK-27610.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 20:59:35 -07:00
Shixiong Zhu 6a317c8f01 [SPARK-27735][SS] Parsing interval string should be case-insensitive in SS
## What changes were proposed in this pull request?

Some APIs in Structured Streaming requires the user to specify an interval. Right now these APIs don't accept upper-case strings.

This PR adds a new method `fromCaseInsensitiveString` to `CalendarInterval` to support paring upper-case strings, and fixes all APIs that need to parse an interval string.

## How was this patch tested?

The new unit test.

Closes #24619 from zsxwing/SPARK-27735.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 13:58:27 -07:00