Skip to content

Conversation

@JoshRosen
Copy link
Contributor

When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC.

This pull request is a minimum-viable-implementation of this idea. In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@SparkQA
Copy link

SparkQA commented May 18, 2015

Test build #32963 has started for PR 6227 at commit b154e86.

@JoshRosen
Copy link
Contributor Author

For https://gist.github.com/680ee530655941defcb2, this patch gives a roughly 3x speedup.

@SparkQA
Copy link

SparkQA commented May 18, 2015

Test build #32963 has finished for PR 6227 at commit b154e86.

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

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/32963/
Test PASSed.

@JoshRosen
Copy link
Contributor Author

Leaving the [WIP] tag on for now while we discuss a few different design decisions.

@JoshRosen
Copy link
Contributor Author

@rxin @zsxwing In the long term, I think that we should consider a more complete design for buffer pooling in our allocator, including thinking through how / whether we want to support pooling for off-heap modes, how we want to match up allocation requests with things in the pages, whether we want to have more manual control over purging pages from the pool, etc. For the immediate 1.4 term, though, I think a super-simple approach like the one in this patch offers a nice improvement. Because the number of different allocation sizes is relatively small (one or two sizes, tops), I think the simple approach is fine for starters.

Copy link
Member

Choose a reason for hiding this comment

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

Just a minor question: I think you want to use LinkedList as Stack since you use pop. Right? If so, here you should use push. push calls addFirst, pop calls removeFirst, while add calls addLast.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Java's linked list is doubly-linked, so I don't think that this makes a perf. difference or anything, which is why I was a little sloppy here.

Copy link
Member

Choose a reason for hiding this comment

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

I thought there was a special reason to use pop :)

@JoshRosen
Copy link
Contributor Author

On closer inspection, I'm thinking that we should probably prefer WeakReferences to SoftReferences, since it's probably better to allow this memory to be released sooner in response to memory demand rather than having the heap grow in order to try to keep empty pages in the pool.

@zsxwing
Copy link
Member

zsxwing commented May 20, 2015

@JoshRosen why not round up size to a power of 2? I think it's more possible to reuse the objects.

@zsxwing
Copy link
Member

zsxwing commented May 20, 2015

+1. WeakReferences is better for cache usage.

@JoshRosen
Copy link
Contributor Author

@zsxwing I think that all of our internal requests are already power-of-2 sized, so I don't think that's a concern yet. Right now, I think we'll only end up allocating pages whose sizes are drawn from a very small set (maybe 4 or fewer standard page sizes, tops). We might consider adding the rounding later, though.

@zsxwing
Copy link
Member

zsxwing commented May 20, 2015

I see. LGTM except the WeakReferences thing.

@JoshRosen JoshRosen changed the title [SPARK-7698] [WIP] Cache and reuse buffers in ExecutorMemoryAllocator when using heap allocation [SPARK-7698] Cache and reuse buffers in ExecutorMemoryAllocator when using heap allocation May 20, 2015
@JoshRosen
Copy link
Contributor Author

Just pushed a commit to fix the WeakReference thing :)

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@SparkQA
Copy link

SparkQA commented May 20, 2015

Test build #33114 has started for PR 6227 at commit fd6cb55.

@zsxwing
Copy link
Member

zsxwing commented May 20, 2015

LGTM

@SparkQA
Copy link

SparkQA commented May 20, 2015

Test build #33114 has finished for PR 6227 at commit fd6cb55.

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

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33114/
Test PASSed.

@rxin
Copy link
Contributor

rxin commented May 20, 2015

LGTM

@JoshRosen
Copy link
Contributor Author

Thanks for the review. I'm going to merge this into master and branch-1.4 (1.4.0).

asfgit pushed a commit that referenced this pull request May 20, 2015
…using heap allocation

When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC.

This pull request is a minimum-viable-implementation of this idea.  In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM.

Author: Josh Rosen <[email protected]>

Closes #6227 from JoshRosen/SPARK-7698 and squashes the following commits:

fd6cb55 [Josh Rosen] SoftReference -> WeakReference
b154e86 [Josh Rosen] WIP sketch of pooling in ExecutorMemoryManager

(cherry picked from commit 7956dd7)
Signed-off-by: Josh Rosen <[email protected]>
@asfgit asfgit closed this in 7956dd7 May 20, 2015
@JoshRosen JoshRosen deleted the SPARK-7698 branch May 20, 2015 23:41
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request May 28, 2015
…using heap allocation

When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC.

This pull request is a minimum-viable-implementation of this idea.  In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM.

Author: Josh Rosen <[email protected]>

Closes apache#6227 from JoshRosen/SPARK-7698 and squashes the following commits:

fd6cb55 [Josh Rosen] SoftReference -> WeakReference
b154e86 [Josh Rosen] WIP sketch of pooling in ExecutorMemoryManager
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request Jun 12, 2015
…using heap allocation

When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC.

This pull request is a minimum-viable-implementation of this idea.  In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM.

Author: Josh Rosen <[email protected]>

Closes apache#6227 from JoshRosen/SPARK-7698 and squashes the following commits:

fd6cb55 [Josh Rosen] SoftReference -> WeakReference
b154e86 [Josh Rosen] WIP sketch of pooling in ExecutorMemoryManager
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
…using heap allocation

When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC.

This pull request is a minimum-viable-implementation of this idea.  In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM.

Author: Josh Rosen <[email protected]>

Closes apache#6227 from JoshRosen/SPARK-7698 and squashes the following commits:

fd6cb55 [Josh Rosen] SoftReference -> WeakReference
b154e86 [Josh Rosen] WIP sketch of pooling in ExecutorMemoryManager
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