Skip to content

Commit dbc1c15

Browse files
Sullivan-PatrickNikhilCollooru
authored andcommitted
Add new error code for queries killed due to too many tasks on cluster
1 parent a7a7f80 commit dbc1c15

File tree

4 files changed

+11
-11
lines changed

4 files changed

+11
-11
lines changed

presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java

+4-5
Original file line numberDiff line numberDiff line change
@@ -49,10 +49,9 @@
4949
import static com.facebook.presto.execution.QueryLimit.createDurationLimit;
5050
import static com.facebook.presto.execution.QueryLimit.getMinimum;
5151
import static com.facebook.presto.spi.StandardErrorCode.ABANDONED_QUERY;
52+
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
5253
import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_TIME_LIMIT;
53-
import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES;
5454
import static com.facebook.presto.spi.StandardErrorCode.SERVER_SHUTTING_DOWN;
55-
import static com.facebook.presto.sql.planner.PlanFragmenterUtils.TOO_MANY_STAGES_MESSAGE;
5655
import static com.google.common.base.Preconditions.checkState;
5756
import static java.lang.String.format;
5857
import static java.util.Comparator.comparingInt;
@@ -296,9 +295,9 @@ void enforceTaskLimits()
296295

297296
while (runningTaskCountAfterKills > maxTotalRunningTaskCountToKillQuery && !taskCountQueue.isEmpty()) {
298297
QueryAndTaskCount<T> queryAndTaskCount = taskCountQueue.poll();
299-
queryAndTaskCount.getQuery().fail(new PrestoException(QUERY_HAS_TOO_MANY_STAGES, format(
300-
"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.",
301-
totalRunningTaskCount, queryAndTaskCount.getTaskCount(), TOO_MANY_STAGES_MESSAGE)));
298+
queryAndTaskCount.getQuery().fail(new PrestoException(CLUSTER_HAS_TOO_MANY_RUNNING_TASKS, format(
299+
"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.",
300+
totalRunningTaskCount, queryAndTaskCount.getTaskCount())));
302301
runningTaskCountAfterKills -= queryAndTaskCount.getTaskCount();
303302
queriesKilledDueToTooManyTask.incrementAndGet();
304303
}

presto-main/src/test/java/com/facebook/presto/execution/TestQueryTrackerHighTaskCountKill.java

+4-4
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424
import java.util.Optional;
2525
import java.util.concurrent.ScheduledExecutorService;
2626

27-
import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES;
27+
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
2828
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
2929
import static org.testng.Assert.assertEquals;
3030
import static org.testng.Assert.assertFalse;
@@ -60,11 +60,11 @@ public void testMultipleQueriesKilledDueToTaskCount()
6060
assertTrue(largeQueryToBeKilled1.getFailureReason().isPresent(), "Query should be killed");
6161
Throwable failureReason1 = largeQueryToBeKilled1.getFailureReason().get();
6262
assertTrue(failureReason1 instanceof PrestoException);
63-
assertEquals(((PrestoException) failureReason1).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
63+
assertEquals(((PrestoException) failureReason1).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
6464
assertTrue(largeQueryToBeKilled2.getFailureReason().isPresent(), "Query should be killed");
6565
Throwable failureReason2 = largeQueryToBeKilled2.getFailureReason().get();
6666
assertTrue(failureReason2 instanceof PrestoException);
67-
assertEquals(((PrestoException) failureReason2).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
67+
assertEquals(((PrestoException) failureReason2).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
6868
}
6969
finally {
7070
scheduledExecutorService.shutdownNow();
@@ -93,7 +93,7 @@ public void testLargeQueryKilledDueToTaskCount_withClusterQueryTracker()
9393
assertFalse(smallQuery.getFailureReason().isPresent(), "small query should not be killed");
9494
Throwable failureReason = largeQueryToBeKilled.getFailureReason().get();
9595
assertTrue(failureReason instanceof PrestoException);
96-
assertEquals(((PrestoException) failureReason).getErrorCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode());
96+
assertEquals(((PrestoException) failureReason).getErrorCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode());
9797
}
9898
finally {
9999
scheduledExecutorService.shutdownNow();

presto-spi/src/main/java/com/facebook/presto/spi/StandardErrorCode.java

+1
Original file line numberDiff line numberDiff line change
@@ -118,6 +118,7 @@ public enum StandardErrorCode
118118
UNSUPPORTED_ANALYZER_TYPE(0x0001_0029, INTERNAL_ERROR),
119119
SLICE_TOO_LARGE(0x0001_002A, INTERNAL_ERROR),
120120
HBO_FAILURE(0x0001_002B, INTERNAL_ERROR),
121+
CLUSTER_HAS_TOO_MANY_RUNNING_TASKS(0x0001_002C, INTERNAL_ERROR),
121122

122123
GENERIC_INSUFFICIENT_RESOURCES(0x0002_0000, INSUFFICIENT_RESOURCES),
123124
EXCEEDED_GLOBAL_MEMORY_LIMIT(0x0002_0001, INSUFFICIENT_RESOURCES),

presto-tests/src/test/java/com/facebook/presto/tests/TestQueryTaskLimit.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@
3434
import static com.facebook.presto.execution.TestQueryRunnerUtil.createQuery;
3535
import static com.facebook.presto.execution.TestQueryRunnerUtil.waitForQueryState;
3636
import static com.facebook.presto.execution.TestQueues.newSession;
37-
import static com.facebook.presto.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES;
37+
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_HAS_TOO_MANY_RUNNING_TASKS;
3838
import static com.facebook.presto.testing.TestingSession.testSessionBuilder;
3939
import static java.util.concurrent.Executors.newCachedThreadPool;
4040
import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -142,7 +142,7 @@ private void waitForQueryToBeKilled(DistributedQueryRunner queryRunner)
142142
for (BasicQueryInfo info : queryRunner.getCoordinator().getQueryManager().getQueries()) {
143143
if (info.getState().isDone()) {
144144
assertNotNull(info.getErrorCode());
145-
assertEquals(info.getErrorCode().getCode(), QUERY_HAS_TOO_MANY_STAGES.toErrorCode().getCode());
145+
assertEquals(info.getErrorCode().getCode(), CLUSTER_HAS_TOO_MANY_RUNNING_TASKS.toErrorCode().getCode());
146146
MILLISECONDS.sleep(100);
147147
return;
148148
}

0 commit comments

Comments
 (0)