Skip to content

Enforce task count limit#13228

Merged
rschlussel merged 2 commits intoprestodb:masterfrom
viczhang861:task-limit
Aug 21, 2019
Merged

Enforce task count limit#13228
rschlussel merged 2 commits intoprestodb:masterfrom
viczhang861:task-limit

Conversation

@viczhang861
Copy link
Contributor

Too many tasks cause stability issues like frequent GC, for query that exceeds an unreasonable high threshold, fail that single query to maintain cluster in a healthy state

Concern: task count check is performed with a delay of one second, is it possible that multiple queries generate a lot tasks (e.g., more than 100K) in less than one second?

Test: tested in production and expensive query is successfully killed

== RELEASE NOTES ==

General Changes
* Add configuration parameters experimental.max-total-running-task-count and experimental.max-query-running-task-count to control the maximal number of tasks for all queries and a single query, respectively

Copy link
Contributor

@aweisberg aweisberg left a comment

Choose a reason for hiding this comment

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

This is a good idea, thanks for making it happen.

This is missing a unit test that demonstrates the policy behaviors of the configuration options are implemented correctly and propagate the desired error message. It would have found the format string issue ;-)

Copy link
Contributor

Choose a reason for hiding this comment

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

The config description mentions this one is soft, so presumably the other is a hard limit? What do soft and hard mean?

Copy link
Contributor

Choose a reason for hiding this comment

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

This won't kill a query if we violated maxTotalRunningTaskCount if highestRunningTaskCount <= maxQueryRunningTaskCount which doesn't fit what I would expect the contract for those configuration values to be?

Is this what you mean by soft limit? They are allowed to exceed maxQueryRunningTaskCount as long as the cluster as a whole doesn't exceed maxTotalRunningTaskCount? This behavior could be clearer from the description.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Correct, description is updated.

Copy link
Contributor

Choose a reason for hiding this comment

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

what's the purpose of the soft limit? If we're out of resources won't we always want to kill something?

Copy link
Contributor

Choose a reason for hiding this comment

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

Can this go negative on some state transitions such as when it goes from PLANNED to anything other then RUNNING?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Very good point, updated this logic to be safe

Tracking task count to identifiy expensive query
@viczhang861 viczhang861 force-pushed the task-limit branch 3 times, most recently from e44e36d to f83e337 Compare August 16, 2019 16:26
@viczhang861 viczhang861 requested a review from aweisberg August 16, 2019 18:51
Copy link
Contributor

@rschlussel rschlussel Aug 19, 2019

Choose a reason for hiding this comment

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

Change the message to be more about the query that was killed, similar to the cluster oom error. Something like:

"Query killed because the cluster is overloaded with too many tasks and this query was running the highest number of tasks. Please try again in a few minutes."

Copy link
Contributor

@aweisberg aweisberg left a comment

Choose a reason for hiding this comment

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

LGTM.

Copy link
Contributor

Choose a reason for hiding this comment

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

what's the purpose of the soft limit? If we're out of resources won't we always want to kill something?

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: typo. excceds -> exceeds

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, if cluster is not busy, let query run.

Copy link
Contributor

Choose a reason for hiding this comment

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

could we do this in the SqlQueryManager where all the other limits (memory, cpu) are enforced?

alternatively, could we do it during scheduling before exceeding the task limit in the first place?

Copy link
Contributor Author

@viczhang861 viczhang861 Aug 20, 2019

Choose a reason for hiding this comment

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

  1. Currently both QueryTracker and SqlQueryManager enforce some limits (time, memory, cpu)
    QueryTracker.java is added later in October 2018,
    https://github.com/prestodb/presto/commits/fe743c0b1836343d428a8d07fba11691cbf29541/presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java
    They both use the same Executor, I assume I should use the new one

  2. @arhimondr mentioned some tasks are dynamically generated. A following task I am investigating is to not create new query execution when cluster is overloaded. This needs more caution thus I want to collect enough production statistics before making a decision about threshold.

Copy link
Contributor

Choose a reason for hiding this comment

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

it sounds like the abstraction between SqlQueryManager and QueryTracker is a bit fuzzy. You can leave it as is for now and we can look at refactoring to make the abstraction clearer.

Copy link
Contributor

Choose a reason for hiding this comment

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

@rschlussel , @viczhang861

QueryTracker.java is added later in October 2018

Correct. Query timeout tracking is refactored into QueryTracker through #11518 . But memory and CPU enforcement is still in SqlQueryManager

Copy link
Contributor

Choose a reason for hiding this comment

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

@wenleix. yeah, I saw that, but it's not clear to me what unites things that go in query tracker vs. SqlQueryManager/what's the purpose of each.

QueryTracker right now does a few things:

  1. keeps track of query info and history and removes queries when they expire
  2. kills queries that exceed a time limit
  3. kills abandoned queries

SqlQueryManager does some other stuff:

  1. creates queries and other lifecycle operations on the query
  2. kills queries that exceed a memory limit
  3. kills queries that exceed a cpu limit
  4. is the thing that calls the query tracker to do its things

Copy link
Contributor

Choose a reason for hiding this comment

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

@rschlussel : Yeah, unfortunately resource management in Presto is a bit scattered right now. Might worthy a refactor to clean up code and to have a better understanding about how it works. cc @oerling , @mbasmanova , @viczhang861 , @bhhari

BTW: isn't memory limit enforcement done in ClusterMemoryManager ?

Copy link
Contributor

Choose a reason for hiding this comment

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

yes, but that's called from the SqlQueryManager.enforceMemoryLimits()

When cluster is overloaded with too many running tasks,
kills the most expensive query whose task count exceeds a
custom-configured threshold
@rschlussel rschlussel merged commit fbf9777 into prestodb:master Aug 21, 2019
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