Commit graph

94 commits

Author SHA1 Message Date
Liang-Chi Hsieh 0042ad575a [SPARK-30290][CORE] Count for merged block when fetching continuous blocks in batch
### What changes were proposed in this pull request?

We added shuffle block fetch optimization in SPARK-9853. In ShuffleBlockFetcherIterator, we merge single blocks into batch blocks. During merging, we should count merged blocks for `maxBlocksInFlightPerAddress`, not original single blocks.

### Why are the changes needed?

If `maxBlocksInFlightPerAddress` is specified, like set it to 1, it should mean one batch block, not one original single block. Otherwise, it will conflict with batch shuffle fetch.

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

No

### How was this patch tested?

Unit test.

Closes #26930 from viirya/respect-max-blocks-inflight.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 18:57:02 +08:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

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

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

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

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
Marcelo Vanzin c5f312a6ac [SPARK-30129][CORE] Set client's id in TransportClient after successful auth
The new auth code was missing this bit, so it was not possible to know which
app a client belonged to when auth was on.

I also refactored the SASL test that checks for this so it also checks the
new protocol (test failed before the fix, passes now).

Closes #26760 from vanzin/SPARK-30129.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-04 17:11:50 -08:00
“attilapiros” fd2bf55aba [SPARK-27651][CORE] Avoid the network when shuffle blocks are fetched from the same host
## What changes were proposed in this pull request?

Before this PR `ShuffleBlockFetcherIterator` was partitioning the block fetches into two distinct sets: local reads and remote fetches. Within this PR (when the feature is enabled by "spark.shuffle.readHostLocalDisk.enabled") a new category is introduced: host-local reads. They are shuffle block fetches where although the block manager is different they are running on the same host along with the requester.

Moreover to get the local directories of the other executors/block managers a new RPC message is introduced `GetLocalDirs` which is sent the the block manager master where it is answered as `BlockManagerLocalDirs`. In `BlockManagerMasterEndpoint` for answering this request the `localDirs` is extracted from the `BlockManagerInfo` and stored separately in a hash map called `executorIdLocalDirs`. Because the earlier used `blockManagerInfo` contains data for the alive block managers (see `org.apache.spark.storage.BlockManagerMasterEndpoint#removeBlockManager`).

Now `executorIdLocalDirs` knows all the local dirs up to the application start (like the external shuffle service does) so in case of an RDD recalculation both host-local shuffle blocks and disk persisted RDD blocks on the same host can be served by reading the files behind the blocks directly.

## How was this patch tested?

### Unit tests

`ExternalShuffleServiceSuite`:
- "SPARK-27651: host local disk reading avoids external shuffle service on the same node"

`ShuffleBlockFetcherIteratorSuite`:
- "successful 3 local reads + 4 host local reads + 2 remote reads"

And with extending existing suites where shuffle metrics was tested.

### Manual tests

Running Spark on YARN in a 4 nodes cluster with 6 executors and having 12 shuffle blocks.

```
$ grep host-local experiment.log
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_2_1, shuffle_0_6_1
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 38 ms
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_0_0, shuffle_0_8_0
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 35 ms
```

Closes #25299 from attilapiros/SPARK-27651.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-26 11:02:25 -08:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

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

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06: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
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
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
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
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
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
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
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
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
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
weixiuli 8b0aa59218 [SPARK-26288][CORE] add initRegisteredExecutorsDB
## What changes were proposed in this pull request?

As we all know that spark on Yarn uses DB https://github.com/apache/spark/pull/7943 to record RegisteredExecutors information which can be reloaded and used again when the ExternalShuffleService is restarted .

The RegisteredExecutors information can't be recorded both in the mode of spark's standalone and spark on k8s , which will cause the RegisteredExecutors information to be lost ,when the ExternalShuffleService is restarted.

To solve the problem above, a method is proposed and is committed .

## How was this patch tested?
new  unit tests

Closes #23393 from weixiuli/SPARK-26288.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-19 16:16:43 -05:00
“attilapiros” 5668c42edf [SPARK-27021][CORE] Cleanup of Netty event loop group for shuffle chunk fetch requests
## What changes were proposed in this pull request?

Creating an Netty `EventLoopGroup` leads to creating a new Thread pool for handling the events. For stopping the threads of the pool the event loop group should be shut down which is properly done for transport servers and clients by calling for example the `shutdownGracefully()` method (for details see the `close()` method of `TransportClientFactory` and `TransportServer`). But there is a separate event loop group for shuffle chunk fetch requests which is in pipeline for handling fetch request (shared between the client and server) and owned by the `TransportContext` and this was never shut down.

## How was this patch tested?

With existing unittest.

This leak is in the production system too but its effect is spiking in the unittest.

Checking the core unittest logs before the PR:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
381
```

And after the PR without whitelisting in thread audit and with an extra `await` after the
` chunkFetchWorkers.shutdownGracefully()`:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
0
```

Closes #23930 from attilapiros/SPARK-27021.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-05 12:31:06 -08:00
Liang-Chi Hsieh cf133e6110 [SPARK-26604][CORE] Clean up channel registration for StreamManager
## What changes were proposed in this pull request?

Now in `TransportRequestHandler.processStreamRequest`, when a stream request is processed, the stream id is not registered with the current channel in stream manager. It should do that so in case of that the channel gets terminated we can remove associated streams of stream requests too.

This also cleans up channel registration in `StreamManager`. Since `StreamManager` doesn't register channel but only `OneForOneStreamManager` does it, this removes `registerChannel` from `StreamManager`. When `OneForOneStreamManager` goes to register stream, it will also register channel for the stream.

## How was this patch tested?

Existing tests.

Closes #23521 from viirya/SPARK-26604.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-16 10:58:07 +08:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
pgandhi 8dd29fe36b [SPARK-25642][YARN] Adding two new metrics to record the number of registered connections as well as the number of active connections to YARN Shuffle Service
Recently, the ability to expose the metrics for YARN Shuffle Service was added as part of [SPARK-18364](https://github.com/apache/spark/pull/22485). We need to add some metrics to be able to determine the number of active connections as well as open connections to the external shuffle service to benchmark network and connection issues on large cluster environments.

Added two more shuffle server metrics for Spark Yarn shuffle service: numRegisteredConnections which indicate the number of registered connections to the shuffle service and numActiveConnections which indicate the number of active connections to the shuffle service at any given point in time.

If these metrics are outputted to a file, we get something like this:

1533674653489 default.shuffleService: Hostname=server1.abc.com, openBlockRequestLatencyMillis_count=729, openBlockRequestLatencyMillis_rate15=0.7110833548897356, openBlockRequestLatencyMillis_rate5=1.657808981793011, openBlockRequestLatencyMillis_rate1=2.2404486061620474, openBlockRequestLatencyMillis_rateMean=0.9242558551196706,
numRegisteredConnections=35,
blockTransferRateBytes_count=2635880512, blockTransferRateBytes_rate15=2578547.6094160094, blockTransferRateBytes_rate5=6048721.726302424, blockTransferRateBytes_rate1=8548922.518223226, blockTransferRateBytes_rateMean=3341878.633637769, registeredExecutorsSize=5, registerExecutorRequestLatencyMillis_count=5, registerExecutorRequestLatencyMillis_rate15=0.0027973949328659836, registerExecutorRequestLatencyMillis_rate5=0.0021278007987206426, registerExecutorRequestLatencyMillis_rate1=2.8270296777387467E-6, registerExecutorRequestLatencyMillis_rateMean=0.006339206380043053, numActiveConnections=35

Closes #22498 from pgandhi999/SPARK-18364.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-21 11:28:33 -08:00
Sean Owen 630e25e355 [SPARK-26026][BUILD] Published Scaladoc jars missing from Maven Central
## What changes were proposed in this pull request?

This restores scaladoc artifact generation, which got dropped with the Scala 2.12 update. The change looks large, but is almost all due to needing to make the InterfaceStability annotations top-level classes (i.e. `InterfaceStability.Stable` -> `Stable`), unfortunately. A few inner class references had to be qualified too.

Lots of scaladoc warnings now reappear. We can choose to disable generation by default and enable for releases, later.

## How was this patch tested?

N/A; build runs scaladoc now.

Closes #23069 from srowen/SPARK-26026.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 08:06:33 -06:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Fokko Driesprong 1a28625355 [SPARK-25408] Move to more ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use of features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No logic has been changed. I think it is important to have a solid codebase with examples that will inspire next PR's to follow up on the best practices.

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

No changes in the logic of Spark, but more in the aesthetics of the code.

## How was this patch tested?

Using the existing unit tests. Since no logic is changed, the existing unit tests should pass.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #22637 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-08 09:58:52 -05:00
Wenchen Fan 5ae20cf1a9 Revert "[SPARK-25408] Move to mode ideomatic Java8"
This reverts commit 44c1e1ab1c.
2018-10-05 11:03:41 +08:00
Fokko Driesprong 44c1e1ab1c [SPARK-25408] Move to mode ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use og features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No code has been changed

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #22399 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-05 02:58:25 +01:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
Sanket Chintapalli ff601cf71d [SPARK-24355] Spark external shuffle server improvement to better handle block fetch requests.
## What changes were proposed in this pull request?

Description:
Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads.
In order to process a client request, it would require one available server netty handler thread.
However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients.
As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process.

This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server.
(Please fill in changes proposed in this fix)

For Original PR please refer here
https://github.com/apache/spark/pull/21402

## How was this patch tested?

Unit tests and stress testing.
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #22173 from redsanket/SPARK-24335.

Authored-by: Sanket Chintapalli <schintap@yahoo-inc.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-09-21 09:05:56 -05:00
Imran Rashid a97001d217 [CORE] Updates to remote cache reads
Covered by tests in DistributedSuite
2018-09-17 14:06:09 -05:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
Imran Rashid 99d2e4e007 [SPARK-24296][CORE] Replicate large blocks as a stream.
When replicating large cached RDD blocks, it can be helpful to replicate
them as a stream, to avoid using large amounts of memory during the
transfer.  This also allows blocks larger than 2GB to be replicated.

Added unit tests in DistributedSuite.  Also ran tests on a cluster for
blocks > 2gb.

Closes #21451 from squito/clean_replication.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-21 11:26:41 -07:00
Misha Dmitriev de4feae3cd [SPARK-24356][CORE] Duplicate strings in File.path managed by FileSegmentManagedBuffer
This patch eliminates duplicate strings that come from the 'path' field of
java.io.File objects created by FileSegmentManagedBuffer. That is, we want
to avoid the situation when multiple File instances for the same pathname
"foo/bar" are created, each with a separate copy of the "foo/bar" String
instance. In some scenarios such duplicate strings may waste a lot of memory
(~ 10% of the heap). To avoid that, we intern the pathname with
String.intern(), and before that we make sure that it's in a normalized
form (contains no "//", "///" etc.) Otherwise, the code in java.io.File
would normalize it later, creating a new "foo/bar" String copy.
Unfortunately, the normalization code that java.io.File uses internally
is in the package-private class java.io.FileSystem, so we cannot call it
here directly.

## What changes were proposed in this pull request?

Added code to ExternalShuffleBlockResolver.getFile(), that normalizes and then interns the pathname string before passing it to the File() constructor.

## How was this patch tested?

Added unit test

Author: Misha Dmitriev <misha@cloudera.com>

Closes #21456 from countmdm/misha/spark-24356.
2018-06-02 23:07:39 -05:00
Xingbo Jiang 8ef167a5f9 [SPARK-24340][CORE] Clean up non-shuffle disk block manager files following executor exits on a Standalone cluster
## What changes were proposed in this pull request?

Currently we only clean up the local directories on application removed. However, when executors die and restart repeatedly, many temp files are left untouched in the local directories, which is undesired behavior and could cause disk space used up gradually.

We can detect executor death in the Worker, and clean up the non-shuffle files (files not ended with ".index" or ".data") in the local directories, we should not touch the shuffle files since they are expected to be used by the external shuffle service.

Scope of this PR is limited to only implement the cleanup logic on a Standalone cluster, we defer to experts familiar with other cluster managers(YARN/Mesos/K8s) to determine whether it's worth to add similar support.

## How was this patch tested?

Add new test suite to cover.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21390 from jiangxb1987/cleanupNonshuffleFiles.
2018-06-01 13:46:05 -07:00
Shixiong Zhu ec63e2d074 [SPARK-23289][CORE] OneForOneBlockFetcher.DownloadCallback.onData should write the buffer fully
## What changes were proposed in this pull request?

`channel.write(buf)` may not write the whole buffer since the underlying channel is a FileChannel, we should retry until the whole buffer is written.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20461 from zsxwing/SPARK-23289.
2018-02-01 21:00:47 +08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
jerryshao 93f92c0ed7 [SPARK-21475][CORE][2ND ATTEMPT] Change to use NIO's Files API for external shuffle service
## What changes were proposed in this pull request?

This PR is the second attempt of #18684 , NIO's Files API doesn't override `skip` method for `InputStream`, so it will bring in performance issue (mentioned in #20119). But using `FileInputStream`/`FileOutputStream` will also bring in memory issue (https://dzone.com/articles/fileinputstream-fileoutputstream-considered-harmful), which is severe for long running external shuffle service. So here in this proposal, only fixing the external shuffle service related code.

## How was this patch tested?

Existing tests.

Author: jerryshao <sshao@hortonworks.com>

Closes #20144 from jerryshao/SPARK-21475-v2.
2018-01-04 11:39:42 -08:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
Shixiong Zhu 14c4a62c12 [SPARK-21475][Core]Revert "[SPARK-21475][CORE] Use NIO's Files API to replace FileInputStream/FileOutputStream in some critical paths"
## What changes were proposed in this pull request?

This reverts commit 5fd0294ff8 because of a huge performance regression.
I manually fixed a minor conflict in `OneForOneBlockFetcher.java`.

`Files.newInputStream` returns `sun.nio.ch.ChannelInputStream`. `ChannelInputStream` doesn't override `InputStream.skip`, so it's using the default `InputStream.skip` which just consumes and discards data. This causes a huge performance regression when reading shuffle files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20119 from zsxwing/revert-SPARK-21475.
2017-12-29 22:33:29 -08:00
Yuming Wang 9df08e218c [SPARK-22454][CORE] ExternalShuffleClient.close() should check clientFactory null
## What changes were proposed in this pull request?

`ExternalShuffleClient.close()` should check `clientFactory` null. otherwise it will throw NPE sometimes:
```
17/11/06 20:08:05 ERROR Utils: Uncaught exception in thread main
java.lang.NullPointerException
	at org.apache.spark.network.shuffle.ExternalShuffleClient.close(ExternalShuffleClient.java:152)
	at org.apache.spark.storage.BlockManager.stop(BlockManager.scala:1407)
	at org.apache.spark.SparkEnv.stop(SparkEnv.scala:89)
	at org.apache.spark.SparkContext$$anonfun$stop$11.apply$mcV$sp(SparkContext.scala:1849)
```

## How was this patch tested?
manual tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19670 from wangyum/SPARK-22454.
2017-11-07 08:30:58 +00:00
jerryshao e1960c3d6f [SPARK-22062][CORE] Spill large block to disk in BlockManager's remote fetch to avoid OOM
## What changes were proposed in this pull request?

In the current BlockManager's `getRemoteBytes`, it will call `BlockTransferService#fetchBlockSync` to get remote block. In the `fetchBlockSync`, Spark will allocate a temporary `ByteBuffer` to store the whole fetched block. This will potentially lead to OOM if block size is too big or several blocks are fetched simultaneously in this executor.

So here leveraging the idea of shuffle fetch, to spill the large block to local disk before consumed by upstream code. The behavior is controlled by newly added configuration, if block size is smaller than the threshold, then this block will be persisted in memory; otherwise it will first spill to disk, and then read from disk file.

To achieve this feature, what I did is:

1. Rename `TempShuffleFileManager` to `TempFileManager`, since now it is not only used by shuffle.
2. Add a new `TempFileManager` to manage the files of fetched remote blocks, the files are tracked by weak reference, will be deleted when no use at all.

## How was this patch tested?

This was tested by adding UT, also manual verification in local test to perform GC to clean the files.

Author: jerryshao <sshao@hortonworks.com>

Closes #19476 from jerryshao/SPARK-22062.
2017-10-17 22:54:38 +08:00
Thomas Graves a74ec6d7bb [SPARK-22218] spark shuffle services fails to update secret on app re-attempts
This patch fixes application re-attempts when running spark on yarn using the external shuffle service with security on.  Currently executors will fail to launch on any application re-attempt when launched on a nodemanager that had an executor from the first attempt.  The reason for this is because we aren't updating the secret key after the first application attempt.  The fix here is to just remove the containskey check to see if it already exists. In this way, we always add it and make sure its the most recent secret.  Similarly remove the check for containsKey on the remove since its just adding extra check that isn't really needed.

Note this worked before spark 2.2 because the check used to be contains (which was looking for the value) rather then containsKey, so that never matched and it was just always adding the new secret.

Patch was tested on a 10 node cluster as well as added the unit test.
The test ran was a wordcount where the output directory already existed.  With the bug present the application attempt failed with max number of executor Failures which were all saslExceptions.  With the fix present the application re-attempts fail with directory already exists or when you remove the directory between attempts the re-attemps succeed.

Author: Thomas Graves <tgraves@unharmedunarmed.corp.ne1.yahoo.com>

Closes #19450 from tgravescs/SPARK-22218.
2017-10-09 12:56:37 -07:00
jerryshao 1da5822e6a [SPARK-21934][CORE] Expose Shuffle Netty memory usage to MetricsSystem
## What changes were proposed in this pull request?

This is a followup work of SPARK-9104 to expose the Netty memory usage to MetricsSystem. Current the shuffle Netty memory usage of `NettyBlockTransferService` will be exposed, if using external shuffle, then the Netty memory usage of `ExternalShuffleClient` and `ExternalShuffleService` will be exposed instead. Currently I don't expose Netty memory usage of `YarnShuffleService`, because `YarnShuffleService` doesn't have `MetricsSystem` itself, and is better to connect to Hadoop's MetricsSystem.

## How was this patch tested?

Manually verified in local cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #19160 from jerryshao/SPARK-21934.
2017-09-21 13:54:30 +08:00