Skip to content

Conversation

@zsxwing
Copy link
Member

@zsxwing zsxwing commented Apr 25, 2017

What changes were proposed in this pull request?

This PR changes Executor's threads to UninterruptibleThread so that we can use runUninterruptibly in CachedKafkaConsumer. However, this is just best effort to avoid hanging forever. If the user usesCachedKafkaConsumer in another thread (e.g., create a new thread or Future), the potential hang may still happen.

How was this patch tested?

The new added test.

// Use UninterruptibleThread to run tasks so that we can allow running codes without being
// interrupted by `Thread.interrupt()`. Some issues, such as KAFKA-1894, HADOOP-10622,
// will hang forever if some methods are interrupted.
new UninterruptibleThread(r, "unused") // thread name will be set by ThreadFactoryBuilder
Copy link
Member Author

Choose a reason for hiding this comment

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

Mostly of the codes are copied from ThreadUtils. This one is the only difference that matters.


case class AvailableOffsetRange(earliest: Long, latest: Long)

private def runUninterruptiblyIfPossiable[T](body: => T): T = Thread.currentThread match {
Copy link
Contributor

Choose a reason for hiding this comment

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

Possiable -> Possible


case class AvailableOffsetRange(earliest: Long, latest: Long)

private def runUninterruptiblyIfPossiable[T](body: => T): T = Thread.currentThread match {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: rename runUninterruptiblyIfPossible

@SparkQA
Copy link

SparkQA commented Apr 25, 2017

Test build #76144 has finished for PR 17761 at commit 3d7fc4d.

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

@SparkQA
Copy link

SparkQA commented Apr 25, 2017

Test build #76145 has finished for PR 17761 at commit bde6120.

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

@mridulm
Copy link
Contributor

mridulm commented Apr 26, 2017

Task interruption is part of our API iirc - kill with interrupt = true.
Doing this will break that contract - did I miss something here ?

Also, IIRC @JoshRosen had done a bunch of work regarding task reaper, etc - which should get affected by this.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 26, 2017

@mridulm this only affect codes calling runUninterruptibly which is not a public API, so won't break any existing codes. The worst case of this PR is some task using CachedKafkaConsumer
needs to wait until network timeout. However, if removing runUninterruptibly added in this PR, it's pretty easy to be stuck in an endless loop of Kafka client when interrupting it. That's wasting resources and unacceptable.

@mridulm
Copy link
Contributor

mridulm commented Apr 26, 2017

@zsxwing Got it, thanks for clarifying.

@tdas
Copy link
Contributor

tdas commented Apr 27, 2017

LGTM. Merging this to master and 2.2

asfgit pushed a commit that referenced this pull request Apr 27, 2017
…x the potential hang in CachedKafkaConsumer

## What changes were proposed in this pull request?

This PR changes Executor's threads to `UninterruptibleThread` so that we can use `runUninterruptibly` in `CachedKafkaConsumer`. However, this is just best effort to avoid hanging forever. If the user uses`CachedKafkaConsumer` in another thread (e.g., create a new thread or Future), the potential hang may still happen.

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <[email protected]>

Closes #17761 from zsxwing/int.

(cherry picked from commit 01c999e)
Signed-off-by: Tathagata Das <[email protected]>
@asfgit asfgit closed this in 01c999e Apr 27, 2017
@zsxwing zsxwing deleted the int branch April 27, 2017 20:59
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