Skip to content

Conversation

@jinxing64
Copy link

@jinxing64 jinxing64 commented Feb 19, 2017

What changes were proposed in this pull request?

Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse.
Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used). The hope is strong especially when migrating data engine to Spark from another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one is very time consuming.
It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than kill the job because of OOM.

In this pr, I propose to fetch big blocks to disk(which is also mentioned in SPARK-3019):

  1. Track average size and also the outliers(which are larger than 2*avgSize) in MapStatus;
  2. Request memory from MemoryManager before fetch blocks and release the memory to MemoryManager when ManagedBuffer is released.
  3. Fetch remote blocks to disk when failing acquiring memory from MemoryManager, otherwise fetch to memory.

This is an improvement for memory control when shuffle blocks and help to avoid OOM in scenarios like below:

  1. Single huge block;
  2. Sizes of many blocks are underestimated in MapStatus and the actual footprint of blocks is much larger than the estimated.

How was this patch tested?

Added unit test in MapStatusSuite and ShuffleBlockFetcherIteratorSuite.

@jinxing64
Copy link
Author

@vanzin @squito
Would you mind to take a look at this when have time ?

@squito
Copy link
Contributor

squito commented Feb 21, 2017

Hi @jinxing64 I posted a comment on jira about the design -- I think this is a big enough change that its worth discussing the design first. Its fine to keep working on the code as a demonstration if you want, but for now I'd ask that you label this a work-in-progress "[WIP]". (I personally have only briefly glanced at the code and am unlikely to look more closely till we sort out the design issues.)

fwiw I think this is will be a great feature, we just need to be thoughtful about it.

@squito
Copy link
Contributor

squito commented Feb 21, 2017

Jenkins, add to whitelist

@SparkQA
Copy link

SparkQA commented Feb 21, 2017

Test build #73224 has finished for PR 16989 at commit 21f6da3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jinxing64
Copy link
Author

@squito
Thanks a lot for your comments : )
Yes, There must be a design doc for discussing. I will prepare and post a pdf to jira.

@jinxing64 jinxing64 changed the title [SPARK-19659] Fetch big blocks to disk when shuffle-read. [WIP][SPARK-19659] Fetch big blocks to disk when shuffle-read. Feb 22, 2017
@jinxing64
Copy link
Author

jinxing64 commented Feb 28, 2017

@rxin @squito @davies @andrewor14 @JoshRosen
I've uploaded a design doc to jira(https://issues.apache.org/jira/browse/SPARK-19659), It's great if you could help comment on it :) Actually, my data warehouse suffers a lot on this issue. Please take a look if possible. Sorry if this is bothering.

@SparkQA
Copy link

SparkQA commented Mar 29, 2017

Test build #75358 has finished for PR 16989 at commit 822b125.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 31, 2017

Test build #75435 has finished for PR 16989 at commit 03213aa.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 31, 2017

Test build #75436 has finished for PR 16989 at commit 65e7c42.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 1, 2017

Test build #75441 has finished for PR 16989 at commit 1ec1c0b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jinxing64 jinxing64 changed the title [WIP][SPARK-19659] Fetch big blocks to disk when shuffle-read. [SPARK-19659] Fetch big blocks to disk when shuffle-read. Apr 1, 2017
@jinxing64 jinxing64 changed the title [SPARK-19659] Fetch big blocks to disk when shuffle-read. [WIP][SPARK-19659] Fetch big blocks to disk when shuffle-read. Apr 10, 2017
@SparkQA
Copy link

SparkQA commented Apr 16, 2017

Test build #75834 has finished for PR 16989 at commit dccd7ff.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jinxing64
Copy link
Author

Jenkins, test this please

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove the protected and make this visible for test.

@SparkQA
Copy link

SparkQA commented Apr 17, 2017

Test build #75855 has finished for PR 16989 at commit 63f059d.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 17, 2017

Test build #75853 has finished for PR 16989 at commit 63f059d.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 17, 2017

Test build #75858 has finished for PR 16989 at commit b6a8993.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 18, 2017

Test build #75883 has finished for PR 16989 at commit c63f39b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 19, 2017

Test build #75935 has finished for PR 16989 at commit 135c668.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 19, 2017

Test build #75938 has finished for PR 16989 at commit 31cfee0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally we should use DiskBlockManager.getFile to store data in file system.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I wanted to use DiskBlockManager.getFile, but I found it's hard to import DiskBlockManager from OneForOneBlockFetcher.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SparkEnv.get.blockManager.diskBlockManager

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan
Yes, but OneForOneBlockFetcher is in network-shuffle package, I find it hard to import SparkEnv from core package. Did I miss something?(I'm sorry if this question is stupid.)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

instead of passing a boolean fetchToDisk, shall we ask the caller to pass in Option<File> file? My concern is that, Spark has a rule about where to write temp files, we can't just write it to the current directory.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan
I understood ~
I will refine, I will replace boolean fetchToDisk with Option<File[]> shuffleFilesOpt.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we remove the partial written file when failing?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that will be good !

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add parameter doc for this

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I will refine.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how is the overhead when serializing hash map with kryo?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we move this into ShuffleBlockFetcherIterator?

Copy link
Author

@jinxing64 jinxing64 May 3, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, ideally this should be moved into ShuffleBlockFetcherIterator, but I didn't find a better implementation other than

      extends MemoryConsumer(tmm, tmm.pageSizeBytes(),
        if (SparkTransportConf.fromSparkConf(SparkEnv.get.conf, "shuffle").preferDirectBufs()) {
          MemoryMode.OFF_HEAP
        } else {
          MemoryMode.ON_HEAP
        }
      )

I think above is not good looking.
And I'd be a little bit hesitant to expose a 'setMode' in MemoryConsumer

@SparkQA
Copy link

SparkQA commented May 25, 2017

Test build #77316 has finished for PR 16989 at commit 3971706.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • public final class FileSegmentManagedBuffer extends ManagedBuffer

@SparkQA
Copy link

SparkQA commented May 25, 2017

Test build #77319 has finished for PR 16989 at commit 188862e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • public final class FileSegmentManagedBuffer extends ManagedBuffer

@SparkQA
Copy link

SparkQA commented May 25, 2017

Test build #77321 has finished for PR 16989 at commit b07a3b6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • public final class FileSegmentManagedBuffer extends ManagedBuffer

@jinxing64 jinxing64 changed the title [WIP][SPARK-19659] Fetch big blocks to disk when shuffle-read. [SPARK-19659] Fetch big blocks to disk when shuffle-read. May 25, 2017

val diskBlockManager = mock(classOf[DiskBlockManager])
doReturn{
var blockId = new TempLocalBlockId(UUID.randomUUID())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: can be val

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.. sorry for nit ...

@cloud-fan
Copy link
Contributor

good job! merging to master/2.2!

asfgit pushed a commit that referenced this pull request May 25, 2017
## What changes were proposed in this pull request?

Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse.
Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used). The hope is strong especially when migrating data engine to Spark from another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one is very time consuming.
It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than kill the job because of OOM.

In this pr, I propose to fetch big blocks to disk(which is also mentioned in SPARK-3019):

1. Track average size and also the outliers(which are larger than 2*avgSize) in MapStatus;
2. Request memory from `MemoryManager` before fetch blocks and release the memory to `MemoryManager` when `ManagedBuffer` is released.
3. Fetch remote blocks to disk when failing acquiring memory from `MemoryManager`, otherwise fetch to memory.

This is an improvement for memory control when shuffle blocks and help to avoid OOM in scenarios like below:
1. Single huge block;
2. Sizes of many blocks are underestimated in `MapStatus` and the actual footprint of blocks is much larger than the estimated.

## How was this patch tested?
Added unit test in `MapStatusSuite` and `ShuffleBlockFetcherIteratorSuite`.

Author: jinxing <[email protected]>

Closes #16989 from jinxing64/SPARK-19659.

(cherry picked from commit 3f94e64)
Signed-off-by: Wenchen Fan <[email protected]>
@asfgit asfgit closed this in 3f94e64 May 25, 2017
@jinxing64
Copy link
Author

@cloud-fan @JoshRosen @mridulm @squito @viirya
Thanks a lot for taking so much time reviewing this patch !
Sorry for the stupid mistakes I made. I will be more careful next time :)

}

// Shuffle remote blocks to disk when the request is too large.
// TODO: Encryption and compression should be considered.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you expand on what the TODO here is? I want to make sure this doesn't slip through the cracks and become forgotten.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually I'm just going to send a follow-up PR. Ideally all local files written by Spark could be encrypted and compressed according to config. One example is UnsafeSorterSpillWriter, it writes data with DiskBlockObjectWriter, which calls SerializerManager.wrapStream and handles encryption and compression automatically.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't really followed this review (sorry), but shuffle data is transmitted encrypted and compressed over the wire, so there might be a chance that there's nothing to do here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah that's a good point! Yea we don't need to encrypt and compress the data again here. I'll update this comment.

One question: do we need to encrypt and compress the data for sort buffer spill and aggregate buffer spill? cc @JoshRosen

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to encrypt and compress the data for sort buffer spill and aggregate buffer spill?

Yes, but I thought I had done that in a previous change. Maybe I missed something.

asfgit pushed a commit that referenced this pull request May 27, 2017
…read

## What changes were proposed in this pull request?

This PR includes some minor improvement for the comments and tests in #16989

## How was this patch tested?

N/A

Author: Wenchen Fan <[email protected]>

Closes #18117 from cloud-fan/follow.

(cherry picked from commit 1d62f8a)
Signed-off-by: Wenchen Fan <[email protected]>
ghost pushed a commit to dbtsai/spark that referenced this pull request May 27, 2017
…read

## What changes were proposed in this pull request?

This PR includes some minor improvement for the comments and tests in apache#16989

## How was this patch tested?

N/A

Author: Wenchen Fan <[email protected]>

Closes apache#18117 from cloud-fan/follow.
asfgit pushed a commit that referenced this pull request Jun 22, 2017
…uffle.

In current code(#16989), big blocks are shuffled to disk.
This pr proposes to collect metrics for remote bytes fetched to disk.

Author: jinxing <[email protected]>

Closes #18249 from jinxing64/SPARK-19937.
return nextChunk;
}

@Override
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jinxing64 this breaks old shuffle service. We should avoid change server side codes. Right now I just disabled this feature in #18467

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, I will try make a pr as soon as possible.

robert3005 pushed a commit to palantir/spark that referenced this pull request Jun 29, 2017
…uffle.

In current code(apache#16989), big blocks are shuffled to disk.
This pr proposes to collect metrics for remote bytes fetched to disk.

Author: jinxing <[email protected]>

Closes apache#18249 from jinxing64/SPARK-19937.

public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
this.targetFile = targetFile;
this.channel = Channels.newChannel(new FileOutputStream(targetFile));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this work with RetryingBlockFetcher? Let's say we have 2 chunks: "chunk 1", "chunk 2". If "chunk 1" fails, it will fail "chunk 2" as well. However, DownloadCallbacks for "chunk 2" are still running. In this case, RetryingBlockFetcher will retry "chunk 2" as well. Hence, there will be 2 DownloadCallbacks writing to the same file.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One possible fix is writing to a temp file and renaming it to the target file.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍
I will make a pr today for this.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zsxwing @cloud-fan
OneForOneBlockFetcher "open blocks" asynchronously. If I understand correctly, the retry of the start() in OneForOneBlockFetcher is only triggered when failure of sending OpenBlocks, but failure of fetching chunk cannot trigger the retry in RetryingBlockFetcher. DownloadCalback is not initialized when the failure of "open blocks" happens. So there cannot be two DownloadCallbacks for same stream working at the same time.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jinxing64 The retry logic is here:

The issue is there will be two DownloadCallbacks download the same content to the same target file. While the first one finishes, ShuffleBlockFetcherIterator may start to read it, however, the second DownloadCallback may be still running and writing to the target file. It could cause ShuffleBlockFetcherIterator reading a partial result.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pardon, I could hardly believe there are two DownloadCallbacks download the same content to the same target file. In my understanding:

  1. When RetryingBlockFetcher retry, there is no DownloadCallback initialized;
  2. When failure of fetching chunk, retry from RetryingBlockFetcher will not be triggered.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zsxwing
Sorry, I just realized this issue. There can be conflict between two DownloadCallbacks. I will figure out a way to resolve this.


@Override
public void onData(String streamId, ByteBuffer buf) throws IOException {
channel.write(buf);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am super-late on reviewing this, apologies, just asking questions for my own understanding, and to consider possible future improvements -- this won't do a zero-copy transfer, will it? That ByteBuffer is still in user space?

From my understanding, we'd need to do special handling to use netty's spliceTo when possible:
https://stackoverflow.com/questions/30322957/is-there-transferfrom-like-functionality-in-netty-for-zero-copy

but I'm still working on putting all the pieces together here and admittedly this is out of my area of expertise

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@squito This is a Java Channel. Not sure how to call io.netty.channel.epoll.AbstractEpollStreamChannel.spliceTo here.

By the way, I think this is a zero-copy transfer since the underlying buffer is an off heap buffer.

Anyway, I found a bug here...

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

right, I realize there isn't a simple one-line change here to switch to using spliceTo, I was wondering what the behavior is.

I actually thought zero-copy and offheap were orthogonal -- anytime netty gives you direct access to bytes, it has to be copied to user space, right?

Copy link
Member

@zsxwing zsxwing Jan 31, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@squito You are right. It needs a copy between user space and kernel space.

cloud-fan added a commit that referenced this pull request Jan 28, 2019
## What changes were proposed in this pull request?

This is a followup of #16989

The fetch-big-block-to-disk feature is disabled by default, because it's not compatible with external shuffle service prior to Spark 2.2. The client sends stream request to fetch block chunks, and old shuffle service can't support it.

After 2 years, Spark 2.2 has EOL, and now it's safe to turn on this feature by default

## How was this patch tested?

existing tests

Closes #23625 from cloud-fan/minor.

Authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
## What changes were proposed in this pull request?

This is a followup of apache#16989

The fetch-big-block-to-disk feature is disabled by default, because it's not compatible with external shuffle service prior to Spark 2.2. The client sends stream request to fetch block chunks, and old shuffle service can't support it.

After 2 years, Spark 2.2 has EOL, and now it's safe to turn on this feature by default

## How was this patch tested?

existing tests

Closes apache#23625 from cloud-fan/minor.

Authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants