Skip to content
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

[SPARK-13398][STREAMING] Move away from thread pool task support to forkjoin #11423

Conversation

holdenk
Copy link
Contributor

@holdenk holdenk commented Feb 29, 2016

What changes were proposed in this pull request?

Remove old deprecated ThreadPoolExecutor and replace with ExecutionContext using a ForkJoinPool. The downside of this is that scala's ForkJoinPool doesn't give us a way to specify the thread pool name (and is a wrapper of Java's in 2.12) except by providing a custom factory. Note that we can't use Java's ForkJoinPool directly in Scala 2.11 since it uses a ExecutionContext which reports system parallelism. One other implicit change that happens is the old ExecutionContext would have reported a different default parallelism since it used system parallelism rather than threadpool parallelism (this was likely not intended but also likely not a huge difference).

The previous version of this PR attempted to use an execution context constructed on the ThreadPool (but not the deprecated ThreadPoolExecutor class) so as to keep the ability to have human readable named threads but this reported system parallelism.

How was this patch tested?

unit tests: streaming/testOnly org.apache.spark.streaming.util.*

@SparkQA
Copy link

SparkQA commented Feb 29, 2016

Test build #52165 has finished for PR 11423 at commit d6b3fb0.

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

@holdenk
Copy link
Contributor Author

holdenk commented Feb 29, 2016

The HiveComptability test seems likely unrelated (happened in another unrelated PR at the same time). Will double check in the morning.

/**
* Extend ForkJoinWorkerThread so we can instantiate it.
*/
class SparkForkJoinWorkerThread(pool: ForkJoinPool) extends ForkJoinWorkerThread(pool) {
Copy link
Member

Choose a reason for hiding this comment

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

Comments are great of course -- javadoc won't do anything here. It could be a line comment.

I think you might be able to collapse most of this with anonymous classes:

    val factory = new ForkJoinWorkerThreadFactory {
      override def newThread(pool: ForkJoinPool) =
        new ForkJoinWorkerThread(pool) {
          setName(prefix + "-" + super.getName)
        }
    }

@SparkQA
Copy link

SparkQA commented Mar 1, 2016

Test build #52201 has finished for PR 11423 at commit 28d7d38.

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

@srowen
Copy link
Member

srowen commented Mar 1, 2016

Jenkins, retest this please

@SparkQA
Copy link

SparkQA commented Mar 1, 2016

Test build #52239 has finished for PR 11423 at commit 28d7d38.

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

@holdenk
Copy link
Contributor Author

holdenk commented Mar 1, 2016

I'll just switch this to use the Scala one for now (this code path does the correct thing in 2.12 but in 2.11 using the Java pool also reports system parallelism).

@holdenk holdenk changed the title [SPARK-13398][STREAMING] Move away from thread pool task support java forkjoin [SPARK-13398][STREAMING] Move away from thread pool task support to forkjoin Mar 3, 2016
@SparkQA
Copy link

SparkQA commented Mar 4, 2016

Test build #52420 has finished for PR 11423 at commit 6e8e7e0.

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

@srowen
Copy link
Member

srowen commented Mar 4, 2016

LGTM

@srowen
Copy link
Member

srowen commented Mar 4, 2016

Merged to master

@asfgit asfgit closed this in c04dc27 Mar 4, 2016
roygao94 pushed a commit to roygao94/spark that referenced this pull request Mar 22, 2016
…orkjoin

## What changes were proposed in this pull request?

Remove old deprecated ThreadPoolExecutor and replace with ExecutionContext using a ForkJoinPool. The downside of this is that scala's ForkJoinPool doesn't give us a way to specify the thread pool name (and is a wrapper of Java's in 2.12) except by providing a custom factory. Note that we can't use Java's ForkJoinPool directly in Scala 2.11 since it uses a ExecutionContext which reports system parallelism. One other implicit change that happens is the old ExecutionContext would have reported a different default parallelism since it used system parallelism rather than threadpool parallelism (this was likely not intended but also likely not a huge difference).

The previous version of this PR attempted to use an execution context constructed on the ThreadPool (but not the deprecated ThreadPoolExecutor class) so as to keep the ability to have human readable named threads but this reported system parallelism.

## How was this patch tested?

unit tests: streaming/testOnly org.apache.spark.streaming.util.*

Author: Holden Karau <holden@us.ibm.com>

Closes apache#11423 from holdenk/SPARK-13398-move-away-from-ThreadPoolTaskSupport-java-forkjoin.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants