Skip to content

[SPARK-22827][CORE] Avoid throwing OutOfMemoryError in case of exception in spill #20014

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed

Conversation

sitalkedia
Copy link

What changes were proposed in this pull request?

Currently, the task memory manager throws an OutofMemory error when there is an IO exception happens in spill() - https://github.com/apache/spark/blob/master/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java#L194. Similarly there any many other places in code when if a task is not able to acquire memory due to an exception we throw an OutofMemory error which kills the entire executor and hence failing all the tasks that are running on that executor instead of just failing one single task.

How was this patch tested?

Unit tests

@sitalkedia
Copy link
Author

Jenkins test this please.

@sitalkedia
Copy link
Author

cc - @rxin, @sameeragarwal, @zsxwing,

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85080 has finished for PR 20014 at commit 91c9257.

  • This patch fails RAT tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • public final class SparkOutOfMemoryError extends OutOfMemoryError

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85081 has finished for PR 20014 at commit 4254d3e.

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

* Instead of throwing {@link OutOfMemoryError}, which kills the executor,
* we should use throw this exception, which will just kill the current task.
*/
public final class SparkOutOfMemoryError extends OutOfMemoryError {
Copy link
Contributor

Choose a reason for hiding this comment

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

is this an internal class? if yes perhaps we should label it.

Copy link
Author

Choose a reason for hiding this comment

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

Yes, this is an internal class. How do you suggest to label it ?

Copy link
Member

Choose a reason for hiding this comment

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

import org.apache.spark.annotation.Private;

@Private

@rxin
Copy link
Contributor

rxin commented Dec 19, 2017

Overall change lgtm.

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85102 has finished for PR 20014 at commit 089ab45.

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

Copy link
Contributor

@jiangxb1987 jiangxb1987 left a comment

Choose a reason for hiding this comment

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

LGTM only one nit, also cc @cloud-fan

/**
* This exception is thrown when a task can not acquire memory from the Memory manager.
* Instead of throwing {@link OutOfMemoryError}, which kills the executor,
* we should use throw this exception, which will just kill the current task.
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: we should throw this exception, which just kills the current task.

import org.apache.spark.unsafe.array.LongArray;
import org.apache.spark.unsafe.memory.MemoryBlock;

import java.io.IOException;
Copy link
Contributor

Choose a reason for hiding this comment

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

java import should comes first, please keep the previous code.

@@ -341,7 +342,7 @@ private void growPointerArrayIfNecessary() throws IOException {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
throw e;
throw new SparkOutOfMemoryError(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Should allocateArray also throw SparkOutOfMemoryError then we can just rethrow it here?

Copy link
Author

Choose a reason for hiding this comment

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

Actually, it already throws SparkOutOfMemoryError, so we can just rethrow here.

@cloud-fan
Copy link
Contributor

LGTM except 2 minor comments

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85115 has finished for PR 20014 at commit 13777de.

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

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85127 has finished for PR 20014 at commit 5682416.

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

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@asfgit asfgit closed this in 3a7494d Dec 20, 2017
asfgit pushed a commit that referenced this pull request Nov 8, 2018
…ggregateExec`, too.

## What changes were proposed in this pull request?

This is a follow-up pr of #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError` is thrown.
We should throw `SparkOutOfMemoryError` in `HashAggregateExec`, too.

## How was this patch tested?

Existing tests.

Closes #22969 from ueshin/issues/SPARK-22827/oome.

Authored-by: Takuya UESHIN <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
asfgit pushed a commit that referenced this pull request Nov 23, 2018
…moryError when catch exception

## What changes were proposed in this pull request?

the pr #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError `is thrown.
so apply for memory using `MemoryConsumer. allocatePage `when  catch exception, use `SparkOutOfMemoryError `instead of `OutOfMemoryError`

## How was this patch tested?
N / A

Closes #23084 from heary-cao/SparkOutOfMemoryError.

Authored-by: caoxuewen <[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
…ggregateExec`, too.

## What changes were proposed in this pull request?

This is a follow-up pr of apache#20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError` is thrown.
We should throw `SparkOutOfMemoryError` in `HashAggregateExec`, too.

## How was this patch tested?

Existing tests.

Closes apache#22969 from ueshin/issues/SPARK-22827/oome.

Authored-by: Takuya UESHIN <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…moryError when catch exception

## What changes were proposed in this pull request?

the pr apache#20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError `is thrown.
so apply for memory using `MemoryConsumer. allocatePage `when  catch exception, use `SparkOutOfMemoryError `instead of `OutOfMemoryError`

## How was this patch tested?
N / A

Closes apache#23084 from heary-cao/SparkOutOfMemoryError.

Authored-by: caoxuewen <[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.

6 participants