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

Add new error code for queries killed due to too many tasks on cluster #24465

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -49,10 +49,9 @@
import static com.facebook.presto.execution.QueryLimit.createDurationLimit;
import static com.facebook.presto.execution.QueryLimit.getMinimum;
import static com.facebook.presto.spi.StandardErrorCode.ABANDONED_QUERY;
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
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.PlanFragmenterUtils.TOO_MANY_STAGES_MESSAGE;
import static com.google.common.base.Preconditions.checkState;
import static java.lang.String.format;
import static java.util.Comparator.comparingInt;
Expand Down Expand Up @@ -296,9 +295,9 @@ void enforceTaskLimits()

while (runningTaskCountAfterKills > maxTotalRunningTaskCountToKillQuery && !taskCountQueue.isEmpty()) {
QueryAndTaskCount<T> queryAndTaskCount = taskCountQueue.poll();
queryAndTaskCount.getQuery().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, queryAndTaskCount.getTaskCount(), TOO_MANY_STAGES_MESSAGE)));
queryAndTaskCount.getQuery().fail(new PrestoException(CLUSTER_HAS_TOO_MANY_RUNNING_TASKS, 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). Please try again later.",
totalRunningTaskCount, queryAndTaskCount.getTaskCount())));
runningTaskCountAfterKills -= queryAndTaskCount.getTaskCount();
queriesKilledDueToTooManyTask.incrementAndGet();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import java.util.Optional;
import java.util.concurrent.ScheduledExecutorService;

import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES;
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -60,11 +60,11 @@ public void testMultipleQueriesKilledDueToTaskCount()
assertTrue(largeQueryToBeKilled1.getFailureReason().isPresent(), "Query should be killed");
Throwable failureReason1 = largeQueryToBeKilled1.getFailureReason().get();
assertTrue(failureReason1 instanceof PrestoException);
assertEquals(((PrestoException) failureReason1).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
assertEquals(((PrestoException) failureReason1).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
assertTrue(largeQueryToBeKilled2.getFailureReason().isPresent(), "Query should be killed");
Throwable failureReason2 = largeQueryToBeKilled2.getFailureReason().get();
assertTrue(failureReason2 instanceof PrestoException);
assertEquals(((PrestoException) failureReason2).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
assertEquals(((PrestoException) failureReason2).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
}
finally {
scheduledExecutorService.shutdownNow();
Expand Down Expand Up @@ -93,7 +93,7 @@ public void testLargeQueryKilledDueToTaskCount_withClusterQueryTracker()
assertFalse(smallQuery.getFailureReason().isPresent(), "small query should not be killed");
Throwable failureReason = largeQueryToBeKilled.getFailureReason().get();
assertTrue(failureReason instanceof PrestoException);
assertEquals(((PrestoException) failureReason).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
assertEquals(((PrestoException) failureReason).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
}
finally {
scheduledExecutorService.shutdownNow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ public enum StandardErrorCode
UNSUPPORTED_ANALYZER_TYPE(0x0001_0029, INTERNAL_ERROR),
SLICE_TOO_LARGE(0x0001_002A, INTERNAL_ERROR),
HBO_FAILURE(0x0001_002B, INTERNAL_ERROR),
CLUSTER_HAS_TOO_MANY_RUNNING_TASKS(0x0001_002C, INTERNAL_ERROR),

GENERIC_INSUFFICIENT_RESOURCES(0x0002_0000, INSUFFICIENT_RESOURCES),
EXCEEDED_GLOBAL_MEMORY_LIMIT(0x0002_0001, INSUFFICIENT_RESOURCES),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
import static com.facebook.presto.execution.TestQueryRunnerUtil.createQuery;
import static com.facebook.presto.execution.TestQueryRunnerUtil.waitForQueryState;
import static com.facebook.presto.execution.TestQueues.newSession;
import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES;
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
import static com.facebook.presto.testing.TestingSession.testSessionBuilder;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
Expand Down Expand Up @@ -142,7 +142,7 @@ private void waitForQueryToBeKilled(DistributedQueryRunner queryRunner)
for (BasicQueryInfo info : queryRunner.getCoordinator().getQueryManager().getQueries()) {
if (info.getState().isDone()) {
assertNotNull(info.getErrorCode());
assertEquals(info.getErrorCode().getCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode().getCode());
assertEquals(info.getErrorCode().getCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode().getCode());
MILLISECONDS.sleep(100);
return;
}
Expand Down
Loading