Skip to content

Conversation

@mgummelt
Copy link

@mgummelt mgummelt commented Feb 23, 2017

What changes were proposed in this pull request?

Increase default refuse_seconds timeout, and make it configurable. See JIRA for details on how this reduces the risk of starvation.

How was this patch tested?

Unit tests, Manual testing, and Mesos/Spark integration test suite

cc @susanxhuynh @skonto @jmlvanre

Copy link
Author

Choose a reason for hiding this comment

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

This is the meat of the functionality change. We call this whenever the state of queuedDrivers or pendingRetryDrivers has changed.

@SparkQA
Copy link

SparkQA commented Feb 23, 2017

Test build #73303 has finished for PR 17031 at commit a16a429.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 23, 2017

Test build #73307 has finished for PR 17031 at commit 42636b9.

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

Copy link
Contributor

@skonto skonto Feb 23, 2017

Choose a reason for hiding this comment

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

Do we still support fine grained?

Copy link
Author

Choose a reason for hiding this comment

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

It's deprecated, but I had to make some changes to it just to compile. I hope to completely remove by 2.2.

Copy link
Contributor

Choose a reason for hiding this comment

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

ok cool!

@skonto
Copy link
Contributor

skonto commented Feb 23, 2017

@mgummelt How the operator could be informed about starvation, shouldn't this be transparent? It could be challenging to watch for corner cases even those with low probability. Are you planning to add the timer-monitoring stuff after this PR? Still reviewing it... need some time...

@mgummelt
Copy link
Author

@skonto It should be clear in the logs. As long as you have at least INFO logs enabled, you'll see "Suppressing offers." in the logs, and little or nothing after, since the offer cycles stop. Unfortunately, Mesos doesn't expose the suppressed state of frameworks, so you can't glean this from state.json.

@mgummelt mgummelt changed the title [SPARK-19702] Add suppress/revive support to the Mesos Spark Dispatcher [SPARK-19702][MESOS] Add suppress/revive support to the Mesos Spark Dispatcher Feb 23, 2017
@susanxhuynh
Copy link
Contributor

The suppress / revive logic LGTM. I didn't look that closely at the refactoring changes. Where are the Mesos/Spark integration tests that you mentioned? @mgummelt

@susanxhuynh
Copy link
Contributor

If we're concerned about the lost reviveOffer() and don't want to handle that corner case, do we want to document it somewhere for operators? "If jobs aren't running and you see [...] in the logs, do this".

@mgummelt
Copy link
Author

@mgummelt
Copy link
Author

@susanxhuynh I don't think it's worth documenting. It should be clear in the logs, which should be where an operator turns if they notice no jobs are launching.

@skonto
Copy link
Contributor

skonto commented Feb 27, 2017

@mgummelt Yes they should look at the logs but how do they know this is something that requires action from their side and not a cluster issue or anything else. It should be documented since it requires manual intervention. Also it makes harder to build recover logic for monitoring systems if they have to dig into logs, I would preferred this to be advertised somewhere like a rest api or something.

The general problem of resource starvation is solved for all other frameworks in the Universe?
I see this solution https://dcosjira.atlassian.net/browse/CASSANDRA-17 for cassandra, should not we have a unified approach? In a SMACK stack, pretty common use case, there will be problems I guess.

Here https://issues.apache.org/jira/browse/MESOS-6112 it is mentioned that we should see the issue with > 5 frameworks, the duplicate https://issues.apache.org/jira/browse/MESOS-3202 refers to a number less than that. What is the minimum setup to reproduce this (how 5 comes up?) and are there any integration tests testing this for spark?

Logic for suppress & revive LGTM.

@skonto
Copy link
Contributor

skonto commented Feb 27, 2017

The only way to fix this generally is to implement some periodic timer that calls reviveOffers() if there are queued/pending drivers to be scheduled. This can be chatty and complicates the code, so I haven't implemented it here.

Shouldn't we only check if we actually get any offers from the master lately and call reviveOffers() only if not? We could have a backoff approach here...

Copy link
Contributor

Choose a reason for hiding this comment

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

Since you are refactoring the code s/url/_.

Copy link
Contributor

Choose a reason for hiding this comment

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

Parentheses are redundant.

Copy link
Author

Choose a reason for hiding this comment

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

see above

Copy link
Contributor

Choose a reason for hiding this comment

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

return is redundant.

Copy link
Contributor

Choose a reason for hiding this comment

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

brackets are redundant.

Copy link
Author

Choose a reason for hiding this comment

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

The brackets are consistent with our other format strings. I'm not trying to refactor all the code in this PR, btw. I just touched the code whose poor style was hindering my ability to solve the problem related to this PR.

@mgummelt
Copy link
Author

Given the concerns about the dispatcher being stuck in a suppressed state, I'm going to solve this a different way. I'm going to increase the default offer decline timeout to 120s and make it configurable, just like it is in the driver. This will make it so that the offer will be offered to 120 other frameworks before circling back to the dispatcher, rather than the default 5. I'll also keep the explicit revive calls when a new driver is submitted or an existing one fails, which immediately causes offers to be re-offered to the dispatcher.

This removes the risk that the driver gets stuck in a suppressed state, because the dispatcher never suppresses itself.

@skonto
Copy link
Contributor

skonto commented Feb 27, 2017

Ok like the Cassandra case you mean right?

@mgummelt
Copy link
Author

@skonto Cassandra supports suppress/revive https://github.com/mesosphere/dcos-cassandra-service/blob/master/cassandra-scheduler/src/main/java/com/mesosphere/dcos/cassandra/scheduler/CassandraScheduler.java#L423

I can't speak for all the frameworks in Universe, Cassandra and Kafka both support suppress revive, and everything built with the DefaultScheduler in dcos-commons gets it for free: https://github.com/mesosphere/dcos-commons/blob/master/sdk/scheduler/src/main/java/com/mesosphere/sdk/scheduler/DefaultScheduler.java#L838

@mgummelt
Copy link
Author

@skonto @susanxhuynh I've updated the solution to use a longer (120s) default refuse timeout, instead of suppressing offers. Please re-review. Just as the previous refuse seconds settings were undocumented, I've left this one undocumented. Users should almost never need to customize it.

@SparkQA
Copy link

SparkQA commented Feb 27, 2017

Test build #73531 has finished for PR 17031 at commit b6e3205.

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

@skonto
Copy link
Contributor

skonto commented Feb 27, 2017

Ok I see. Cassandra uses 30. What is a reasonable timeout?

@mgummelt
Copy link
Author

It depends on the application. It's the amount of time you have to wait before having the opportunity to use those resources again. But if you explicitly revive, which we do here whenever we need more resources, then it doesn't matter. We could set it to infinity and still never be starved, because we'll always get another shot at the resources when we revive.

@mgummelt mgummelt changed the title [SPARK-19702][MESOS] Add suppress/revive support to the Mesos Spark Dispatcher [SPARK-19702][MESOS] Increase default refuse_seconds timeout in the Mesos Spark Dispatcher Feb 27, 2017
@skonto
Copy link
Contributor

skonto commented Mar 1, 2017

@mgummelt Here is my rationale about the refuse time. As stated here:
https://issues.apache.org/jira/browse/MESOS-3202 and given the timeout for cassandra, I have at most 30 seconds for some other framework to accept resources in the framework list, otherwise the first one will be asked again. So implicitly along with the master delay for making offers, this value limits the number of frameworks that will be asked for the offer which was declined by cassandra (assuming cassandra is the first framework in the list). So if you have many frameworks in that list at last some will starve. So refuse offer should has a large value to give the opportunity more frameworks to be asked for the offer. We need to break that loop right? Am I missing something here?

@mgummelt
Copy link
Author

mgummelt commented Mar 1, 2017

Your understanding is correct. You must set refuse_seconds for all your frameworks to some value N, such that N >= #frameworks. So for this change, if some operator is running >120 frameworks, they may need to configure this value. However, I'm not aware of any Mesos cluster on Earth running that many frameworks.

@mgummelt
Copy link
Author

mgummelt commented Mar 2, 2017

@skonto Any other concerns? Can I get a LGTM?

@skonto
Copy link
Contributor

skonto commented Mar 3, 2017

@mgummelt LGTM. Thanks for the clarifications. Btw I would expect N to be a mesos cluster config option because this is a global issue/workaround. @srowen could we get a merge pls?

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

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

This looks like a large amount of change relative to the description. Is this intentional, all the incidental code changes?

@skonto
Copy link
Contributor

skonto commented Mar 3, 2017

@srowen There are parts for refactoring only purposes, improving quality.

@mgummelt
Copy link
Author

mgummelt commented Mar 3, 2017

@srowen Yes, most of the code is refactoring that I came across when solving this. If that's going to delay this being merged, please let me know and I can remove the refactoring.

@mgummelt
Copy link
Author

mgummelt commented Mar 3, 2017

@skonto I completely agree that this is a cluster-wide issue, but unfortunately that's the state of things. In the long-term, optimistic offers in Mesos should fix this.

@mgummelt mgummelt force-pushed the SPARK-19702-suppress-revive branch from b6e3205 to ba864d0 Compare March 4, 2017 01:53
@mgummelt
Copy link
Author

mgummelt commented Mar 4, 2017

@srowen Just to move things along, I removed everything not directly relevant to this JIRA.

@SparkQA
Copy link

SparkQA commented Mar 4, 2017

Test build #73883 has finished for PR 17031 at commit ba864d0.

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

@srowen
Copy link
Member

srowen commented Mar 4, 2017

Compared to the title, this looks like a significant change, still. Is the intent something different from the JIRA? this doens't just increase a default. I don't have any opinion on the changes, just commenting on the consistency of change vs discussion and paper trail

@skonto
Copy link
Contributor

skonto commented Mar 4, 2017

@mgummelt do we want to keep the suppress/revive technique, time-out increase is not enough? I think that is the added code here compared to what someone expects from the title.
In the jira it says:

We must implement increase the refuse_seconds timeout to solve this problem. Another option would have been to implement suppress/revive, but that can cause starvation due to the unreliability of mesos RPC calls.

Also the description must be updated IMHO.

@mgummelt
Copy link
Author

mgummelt commented Mar 6, 2017

@srowen To support increasing the default, I've had to:

  • make refuse_seconds configurable
  • factor out declineOffer so the dispatcher can use it in addition to the coarse grained scheduler.
  • persist the schedulerDriver in both the dispatcher scheduler and the coarse grained scheduler so we can access it in callbacks that aren't passed the driver object.

@mgummelt
Copy link
Author

mgummelt commented Mar 6, 2017

@skonto updated the description.

@SparkQA
Copy link

SparkQA commented Mar 6, 2017

Test build #74020 has finished for PR 17031 at commit b5fb61e.

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

@mgummelt
Copy link
Author

mgummelt commented Mar 7, 2017

@srowen ping

@srowen
Copy link
Member

srowen commented Mar 7, 2017

Merged to master

@asfgit asfgit closed this in 2e30c0b Mar 7, 2017
@mgummelt
Copy link
Author

mgummelt commented Mar 7, 2017

Thanks!

@mgummelt mgummelt deleted the SPARK-19702-suppress-revive branch March 7, 2017 21:39
mgummelt pushed a commit to d2iq-archive/spark that referenced this pull request Mar 7, 2017
…esos Spark Dispatcher

Increase default refuse_seconds timeout, and make it configurable.  See JIRA for details on how this reduces the risk of starvation.

Unit tests, Manual testing, and Mesos/Spark integration test suite

cc susanxhuynh skonto jmlvanre

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes apache#17031 from mgummelt/SPARK-19702-suppress-revive.
mgummelt pushed a commit to d2iq-archive/spark that referenced this pull request Jun 8, 2017
…esos Spark Dispatcher

Increase default refuse_seconds timeout, and make it configurable.  See JIRA for details on how this reduces the risk of starvation.

Unit tests, Manual testing, and Mesos/Spark integration test suite

cc susanxhuynh skonto jmlvanre

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes apache#17031 from mgummelt/SPARK-19702-suppress-revive.
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