Enforce task count limit#13228
Conversation
aweisberg
left a comment
There was a problem hiding this comment.
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 ;-)
There was a problem hiding this comment.
The config description mentions this one is soft, so presumably the other is a hard limit? What do soft and hard mean?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Correct, description is updated.
There was a problem hiding this comment.
what's the purpose of the soft limit? If we're out of resources won't we always want to kill something?
presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java
Outdated
Show resolved
Hide resolved
presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java
Outdated
Show resolved
Hide resolved
presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Can this go negative on some state transitions such as when it goes from PLANNED to anything other then RUNNING?
There was a problem hiding this comment.
Very good point, updated this logic to be safe
presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java
Outdated
Show resolved
Hide resolved
Tracking task count to identifiy expensive query
e44e36d to
f83e337
Compare
There was a problem hiding this comment.
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."
f83e337 to
c8583ed
Compare
There was a problem hiding this comment.
what's the purpose of the soft limit? If we're out of resources won't we always want to kill something?
There was a problem hiding this comment.
nit: typo. excceds -> exceeds
There was a problem hiding this comment.
Yes, if cluster is not busy, let query run.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
-
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 -
@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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
@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:
- keeps track of query info and history and removes queries when they expire
- kills queries that exceed a time limit
- kills abandoned queries
SqlQueryManager does some other stuff:
- creates queries and other lifecycle operations on the query
- kills queries that exceed a memory limit
- kills queries that exceed a cpu limit
- is the thing that calls the query tracker to do its things
There was a problem hiding this comment.
@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 ?
There was a problem hiding this comment.
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
c8583ed to
ff49687
Compare
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