Skip to content

Conversation

@zsxwing
Copy link
Member

@zsxwing zsxwing commented Jan 18, 2019

What changes were proposed in this pull request?

ByteBuffer.allocate may throw OutOfMemoryError when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its BlockFetchingListener. onBlockFetchSuccess doesn't complete Promise.

This PR catches Throwable and uses the error to complete Promise.

How was this patch tested?

Added a unit test. Since I cannot make ByteBuffer.allocate throw OutOfMemoryError, I passed a negative size to make ByteBuffer.allocate fail. Although the error type is different, it should trigger the same code path.

@zsxwing zsxwing changed the title Fix a bug that BlockTransferService.fetchBlockSync may hang forever [SPARK-26665][Core]Fix a bug that BlockTransferService.fetchBlockSync may hang forever Jan 18, 2019
@zsxwing zsxwing requested a review from cloud-fan January 18, 2019 22:44
@SparkQA
Copy link

SparkQA commented Jan 19, 2019

Test build #101413 has finished for PR 23590 at commit da1aaac.

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

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA
Copy link

SparkQA commented Jan 19, 2019

Test build #101427 has finished for PR 23590 at commit da1aaac.

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

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA
Copy link

SparkQA commented Jan 20, 2019

Test build #101437 has finished for PR 23590 at commit da1aaac.

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

ret.flip()
result.success(new NioManagedBuffer(ret))
} catch {
case e: Throwable => result.failure(e)
Copy link
Contributor

Choose a reason for hiding this comment

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

An alternative to catching and ignoring Throwable (after result.failures) is to use a boolean flag to indicate if result.success was signalled - and if not, invoke result.failure in finally.

var success = false
try {
  ...
  result.success(...)
  success = true
} finally {
  if (! success) result.failure(e)
}

Copy link
Contributor

Choose a reason for hiding this comment

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

we need to get the e, I think the catch is necessary here.

Copy link
Contributor

@mridulm mridulm Jan 21, 2019

Choose a reason for hiding this comment

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

Good point - it is required for the failure arg.
But I dont like the throwable being ignored in the catch once failure is signalled.
For example, interruption, OOM, thread death, etc - we should rethrow it once promise is updated.

Copy link
Contributor

@cloud-fan cloud-fan Jan 21, 2019

Choose a reason for hiding this comment

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

it's not ignored, the exception will be rethrown in ThreadUtils.awaitResult at the end of fetchBlockSync. Maybe we should not use Promise, and do it manually to be more explicit.

Copy link
Contributor

Choose a reason for hiding this comment

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

Looks like this is a common anti pattern in spark; the issue is basically thread/pool specific exceptions start leaking to other threads.
Though my basic concern was regarding things like OOM, and they do get propagated - so I guess we can live with this for now.

Copy link
Member Author

Choose a reason for hiding this comment

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

Promise is the most convenient way to turn an async listener to a Scala Future in order to call blocking APIs provided by Future.

Copy link
Member Author

Choose a reason for hiding this comment

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

Yeah, that's propagated and the code calling fetchBlockSync will handle it.

listener: BlockFetchingListener,
tempFileManager: DownloadFileManager): Unit = {
// Notify BlockFetchingListener with a bad ManagedBuffer asynchronously
new Thread() {
Copy link
Contributor

Choose a reason for hiding this comment

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

why do we create a thread here?

Copy link
Member Author

Choose a reason for hiding this comment

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

Otherwise, BlockFetchingListener will be called in the same thread and fail at once before Thread.awaitResult is called. Then the test won't be able to reproduce the issue.

@cloud-fan
Copy link
Contributor

good catch! LGTM except a comment for the test.

@zsxwing
Copy link
Member Author

zsxwing commented Jan 22, 2019

Thanks! Merging to master and 2.4.

asfgit pushed a commit that referenced this pull request Jan 22, 2019
…c may hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

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

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <[email protected]>
Signed-off-by: Shixiong Zhu <[email protected]>
(cherry picked from commit 66450bb)
Signed-off-by: Shixiong Zhu <[email protected]>
@asfgit asfgit closed this in 66450bb Jan 22, 2019
@zsxwing zsxwing deleted the SPARK-26665 branch January 22, 2019 17:10
@dongjoon-hyun
Copy link
Member

Hi, @zsxwing , @cloud-fan , @mridulm .
Since the underlying code seems to be added by SPARK-22062 at Spark 2.3.0, I'm wondering if we can have this at branch-2.3.
cc @maropu since he is a release manager for 2.3.3.

@zsxwing
Copy link
Member Author

zsxwing commented Jan 22, 2019

@dongjoon-hyun I think SPARK-22062 just moved codes. This is a long standing issue.

@dongjoon-hyun
Copy link
Member

Oh, thank you for the info. Then, can we have this to branch-2.3?

asfgit pushed a commit that referenced this pull request Jan 22, 2019
…c may hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

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

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <[email protected]>
Signed-off-by: Shixiong Zhu <[email protected]>
@zsxwing
Copy link
Member Author

zsxwing commented Jan 22, 2019

@dongjoon-hyun Sure. This is a pretty safe change. I just pushed to branch-2.3.

@dongjoon-hyun
Copy link
Member

Great! Thank you so much, @zsxwing .

jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…c may hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

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

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes apache#23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <[email protected]>
Signed-off-by: Shixiong Zhu <[email protected]>
srowen pushed a commit that referenced this pull request Jun 30, 2019
…checked exception missed

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

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 <[email protected]>
Co-authored-by: lajin <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit 0e42100)
Signed-off-by: Sean Owen <[email protected]>
srowen pushed a commit that referenced this pull request Jun 30, 2019
…checked exception missed

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

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 <[email protected]>
Co-authored-by: lajin <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit 0e42100)
Signed-off-by: Sean Owen <[email protected]>
Tonix517 pushed a commit to Tonix517/spark that referenced this pull request Jul 3, 2019
…checked exception missed

## What changes were proposed in this pull request?

This is very like apache#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 apache#24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <[email protected]>
Co-authored-by: lajin <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
kai-chi pushed a commit to kai-chi/spark that referenced this pull request Jul 23, 2019
…c may hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

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

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes apache#23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <[email protected]>
Signed-off-by: Shixiong Zhu <[email protected]>
(cherry picked from commit 66450bb)
Signed-off-by: Shixiong Zhu <[email protected]>
kai-chi pushed a commit to kai-chi/spark that referenced this pull request Aug 1, 2019
…c may hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

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

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes apache#23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <[email protected]>
Signed-off-by: Shixiong Zhu <[email protected]>
(cherry picked from commit 66450bb)
Signed-off-by: Shixiong Zhu <[email protected]>
rluta pushed a commit to rluta/spark that referenced this pull request Sep 17, 2019
…checked exception missed

This is very like apache#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`.

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 apache#24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <[email protected]>
Co-authored-by: lajin <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit 0e42100)
Signed-off-by: Sean Owen <[email protected]>
kai-chi pushed a commit to kai-chi/spark that referenced this pull request Sep 26, 2019
…checked exception missed

This is very like apache#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`.

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 apache#24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <[email protected]>
Co-authored-by: lajin <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit 0e42100)
Signed-off-by: Sean Owen <[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.

5 participants