-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Enforce task count limit #13228
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
Enforce task count limit #13228
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,12 +35,16 @@ | |
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.ScheduledFuture; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
|
|
||
| import static com.facebook.presto.SystemSessionProperties.getQueryMaxExecutionTime; | ||
| import static com.facebook.presto.SystemSessionProperties.getQueryMaxRunTime; | ||
| import static com.facebook.presto.spi.StandardErrorCode.ABANDONED_QUERY; | ||
| import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_TIME_LIMIT; | ||
| import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES; | ||
| import static com.facebook.presto.spi.StandardErrorCode.SERVER_SHUTTING_DOWN; | ||
| import static com.facebook.presto.sql.planner.PlanFragmenter.TOO_MANY_STAGES_MESSAGE; | ||
| import static com.google.common.base.Preconditions.checkState; | ||
| import static java.lang.String.format; | ||
| import static java.util.Objects.requireNonNull; | ||
|
|
@@ -51,6 +55,12 @@ public class QueryTracker<T extends TrackedQuery> | |
| private static final Logger log = Logger.get(QueryTracker.class); | ||
|
|
||
| private final int maxQueryHistory; | ||
| private final int maxTotalRunningTaskCount; | ||
| private final int maxQueryRunningTaskCount; | ||
|
|
||
| private final AtomicInteger runningTaskCount = new AtomicInteger(); | ||
| private final AtomicLong queriesKilledDueToTooManyTask = new AtomicLong(); | ||
|
|
||
| private final Duration minQueryExpireAge; | ||
|
|
||
| private final ConcurrentMap<QueryId, T> queries = new ConcurrentHashMap<>(); | ||
|
|
@@ -69,6 +79,8 @@ public QueryTracker(QueryManagerConfig queryManagerConfig, ScheduledExecutorServ | |
| this.minQueryExpireAge = queryManagerConfig.getMinQueryExpireAge(); | ||
| this.maxQueryHistory = queryManagerConfig.getMaxQueryHistory(); | ||
| this.clientTimeout = queryManagerConfig.getClientTimeout(); | ||
| this.maxTotalRunningTaskCount = queryManagerConfig.getMaxTotalRunningTaskCount(); | ||
| this.maxQueryRunningTaskCount = queryManagerConfig.getMaxQueryRunningTaskCount(); | ||
|
|
||
| this.queryManagementExecutor = requireNonNull(queryManagementExecutor, "queryManagementExecutor is null"); | ||
| } | ||
|
|
@@ -91,6 +103,15 @@ public synchronized void start() | |
| log.error(e, "Error enforcing query timeout limits"); | ||
| } | ||
|
|
||
| try { | ||
| if (maxTotalRunningTaskCount != Integer.MAX_VALUE && maxQueryRunningTaskCount != Integer.MAX_VALUE) { | ||
| enforceTaskLimits(); | ||
| } | ||
| } | ||
| catch (Throwable e) { | ||
| log.error(e, "Error enforcing running task limits"); | ||
| } | ||
|
|
||
| try { | ||
| removeExpiredQueries(); | ||
| } | ||
|
|
@@ -167,6 +188,16 @@ public void expireQuery(QueryId queryId) | |
| .ifPresent(expirationQueue::add); | ||
| } | ||
|
|
||
| public long getRunningTaskCount() | ||
| { | ||
| return runningTaskCount.get(); | ||
| } | ||
|
|
||
| public long getQueriesKilledDueToTooManyTask() | ||
| { | ||
| return queriesKilledDueToTooManyTask.get(); | ||
| } | ||
|
|
||
| /** | ||
| * Enforce query max runtime/execution time limits | ||
| */ | ||
|
|
@@ -189,6 +220,39 @@ private void enforceTimeLimits() | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * When cluster reaches max tasks limit and also a single query | ||
| * exceeds a threshold, kill this query | ||
| */ | ||
| private void enforceTaskLimits() | ||
| { | ||
| int totalRunningTaskCount = 0; | ||
| int highestRunningTaskCount = 0; | ||
| Optional<T> highestRunningTaskQuery = Optional.empty(); | ||
| for (T query : queries.values()) { | ||
| if (query.isDone()) { | ||
| continue; | ||
| } | ||
| int runningTaskCount = query.getRunningTaskCount(); | ||
| totalRunningTaskCount += runningTaskCount; | ||
| if (runningTaskCount > highestRunningTaskCount) { | ||
| highestRunningTaskCount = runningTaskCount; | ||
| highestRunningTaskQuery = Optional.of(query); | ||
| } | ||
| } | ||
|
|
||
| runningTaskCount.set(totalRunningTaskCount); | ||
|
|
||
| if (totalRunningTaskCount > maxTotalRunningTaskCount && | ||
| highestRunningTaskCount > maxQueryRunningTaskCount && | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This won't kill a query if we violated Is this what you mean by soft limit? They are allowed to exceed
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Correct, description is updated.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? |
||
| highestRunningTaskQuery.isPresent()) { | ||
| highestRunningTaskQuery.get().fail(new PrestoException(QUERY_HAS_TOO_MANY_STAGES, format( | ||
| "Query killed because the cluster is overloaded with too many tasks (%s) and this query was running with the highest number of tasks (%s). %s Otherwise, please try again later.", | ||
| totalRunningTaskCount, highestRunningTaskCount, TOO_MANY_STAGES_MESSAGE))); | ||
| queriesKilledDueToTooManyTask.incrementAndGet(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Prune extraneous info from old queries | ||
| */ | ||
|
|
@@ -292,6 +356,8 @@ public interface TrackedQuery | |
|
|
||
| Optional<DateTime> getEndTime(); | ||
|
|
||
| int getRunningTaskCount(); | ||
|
|
||
| void fail(Throwable cause); | ||
|
|
||
| // XXX: This should be removed when the client protocol is improved, so that we don't need to hold onto so much query history | ||
|
|
||
There was a problem hiding this comment.
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?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@rschlussel , @viczhang861
Correct. Query timeout tracking is refactored into
QueryTrackerthrough #11518 . But memory and CPU enforcement is still inSqlQueryManagerThere was a problem hiding this comment.
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:
SqlQueryManager does some other stuff:
There was a problem hiding this comment.
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?There was a problem hiding this comment.
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()