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

STORM-3040: Improve scheduler performance #2647

Merged
merged 3 commits into from
May 14, 2018
Merged

Conversation

revans2
Copy link
Contributor

@revans2 revans2 commented Apr 26, 2018

There are a lot of different scheduler improvements. Mostly these are either caching, storing data in multiple ways so we can look it up quickly, and finally lazily sorting nodes in a rack only when it is needed, instead of all ahead of time.

I also added in performance tests. They currently pass on travis, but I would like to hear from others on if this solution looks good or if there is a better way for us to do performance testing.

@danny0405
Copy link

@revans2
In total, this is a good promotion direction.

So this promotion mainly focus on repetitive computation and some eagerly computed data structure, right?

What is the average promotion percentage when applied this patch?

Copy link
Contributor

@kishorvpatil kishorvpatil left a comment

Choose a reason for hiding this comment

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

👍 LGTM

@revans2
Copy link
Contributor Author

revans2 commented May 4, 2018

@danny0405 In my tests TestResourceAwareScheduler.testLargeTopologiesCommon went from about 7 mins to about 7 seconds. For TestResourceAwareScheduler.testLargeTopologiesOnLargeClustersGras I don't have a before value because I killed it after an hour. The after is about 7 seconds per topology, or about a min and a half.

@revans2
Copy link
Contributor Author

revans2 commented May 7, 2018

@danny0405 @kishorvpatil With some recent changes to master my patch started to fail with some checkstyle issues. I have rebased and fixed all of the issues. Please take a look again, specifically the second commit and let me know.

@revans2
Copy link
Contributor Author

revans2 commented May 7, 2018

Oh I forgot I also added back in something I messed up before and added back in anti-affinity to GRAS.

@danny0405
Copy link

@revans2
I approve with you promotion totally.

The only concern are all kinds of cache we use here, now storm has many caches not just for scheduling. I just think if we can make a disk-storage-backend for all of such caches master needs. Disk cache has better fault-tolerance and is much cheeper than memory, but this is another promotion direction, and has nothing to do with this patch.

BTW: thx for your nice work.

@@ -48,6 +49,9 @@

public class Cluster implements ISchedulingState {
private static final Logger LOG = LoggerFactory.getLogger(Cluster.class);
private static final Function<String, Set<WorkerSlot>> MAKE_SET = (x) -> new HashSet<>();
private static final Function<String, Map<WorkerSlot, NormalizedResourceRequest>> MAKE_MAP = (x) -> new HashMap<>();

Choose a reason for hiding this comment

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

HashSet is ok for now single daemon scheduling, we may make it thread safe when we want to support parallel scheduling, so can we add a comment about thread safe here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I am happy to add in a comment to Cluster itself about it, as none of Cluster is currently thread safe.

As for parallel scheduling the plan that we had been thinking about was more around scheduling multiple topologies in parallel, rather then trying to make a single scheduler strategy multi-threaded, but both have advantages and disadvantages.

@@ -763,6 +773,7 @@ public void setAssignments(
assertValidTopologyForModification(assignment.getTopologyId());
}
assignments.clear();
totalResourcesPerNodeCache.clear();

Choose a reason for hiding this comment

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

Actually we can reuse these cache for the next scheduling round, when we bring in central master cache in.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I tried that, but it didn't have the performance boost I was hoping for. The vast majority of the performance problem came from recomputing the value each time we wanted to sort, with for GRAS is once per executor. So without the cache for a large topology we were recomputing things hundreds of thousands of times. With the cache it is only how many nodes are in the cluster, which ends up being relatively small. In reality the noise between runs drowned out any improvement, so I opted to not do the change.

Copy link
Contributor

@kishorvpatil kishorvpatil left a comment

Choose a reason for hiding this comment

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

Still 👍

Copy link
Contributor

@Ethanlm Ethanlm left a comment

Choose a reason for hiding this comment

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

👍

@revans2
Copy link
Contributor Author

revans2 commented May 8, 2018

@danny0405 I added in the comments about thread safety like you suggested.

Copy link
Contributor

@Ethanlm Ethanlm left a comment

Choose a reason for hiding this comment

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

+1

@danny0405
Copy link

@revans2
Thx for your work, the storm-core travis check still fails, we should fix that.

@revans2
Copy link
Contributor Author

revans2 commented May 9, 2018

@danny0405 the failure is a known race condition around netty and is not related to this change.

@asfgit asfgit merged commit 558e9b6 into apache:master May 14, 2018
asfgit pushed a commit that referenced this pull request May 14, 2018
… into STORM-3040

STORM-3040: Improve scheduler performance

This closes #2647
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