diff --git a/.travis.yml b/.travis.yml
index b79315edc54e5..4232e538d3a25 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -18,7 +18,7 @@ env:
- TEST_SPECIFIC_MODULES=presto-cassandra
- TEST_SPECIFIC_MODULES=presto-hive
- TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-materialized"
- - TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-recoverable-grouped-execution"
+ - TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-recoverable-execution"
- TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-pushdown-filter-queries-basic"
- TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-pushdown-filter-queries-advanced"
- TEST_SPECIFIC_MODULES=presto-hive TEST_FLAGS="-P test-hive-repartitioning"
diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml
index 591e06d25d2ec..bef0409841bdb 100644
--- a/presto-hive/pom.xml
+++ b/presto-hive/pom.xml
@@ -424,7 +424,7 @@
**/TestHiveDistributedAggregationsWithExchangeMaterialization.java
**/TestHiveDistributedQueriesWithExchangeMaterialization.java
**/TestHiveDistributedQueriesWithOptimizedRepartitioning.java
- **/TestHiveRecoverableGroupedExecution.java
+ **/TestHiveRecoverableExecution.java
**/TestHivePushdownFilterQueries.java
**/TestHivePushdownIntegrationSmokeTest.java
**/TestHivePushdownDistributedQueries.java
@@ -485,7 +485,7 @@
- test-hive-recoverable-grouped-execution
+ test-hive-recoverable-execution
@@ -493,7 +493,7 @@
maven-surefire-plugin
- **/TestHiveRecoverableGroupedExecution.java
+ **/TestHiveRecoverableExecution.java
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePushdownFilterQueries.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePushdownFilterQueries.java
index 4cc6de6b71e34..657e316da56e6 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePushdownFilterQueries.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePushdownFilterQueries.java
@@ -103,6 +103,9 @@ private static QueryRunner createQueryRunner()
{
DistributedQueryRunner queryRunner = HiveQueryRunner.createQueryRunner(getTables(),
ImmutableMap.of("experimental.pushdown-subfields-enabled", "true"),
+ // TODO: enable failure detector. Currently this test has a ton of major GC activity on travis,
+ // and the failure detector may make the test run longer
+ ImmutableMap.of("failure-detector.enabled", "false"),
"sql-standard",
ImmutableMap.of("hive.pushdown-filter-enabled", "true"),
Optional.empty());
@@ -173,8 +176,8 @@ public void testLegacyUnnest()
assertQuery(legacyUnnest, "SELECT orderkey, date.day FROM lineitem_ex CROSS JOIN UNNEST(dates) t(date)",
"SELECT orderkey, day(shipdate) FROM lineitem WHERE orderkey % 31 <> 0 UNION ALL " +
- "SELECT orderkey, day(commitdate) FROM lineitem WHERE orderkey % 31 <> 0 UNION ALL " +
- "SELECT orderkey, day(receiptdate) FROM lineitem WHERE orderkey % 31 <> 0");
+ "SELECT orderkey, day(commitdate) FROM lineitem WHERE orderkey % 31 <> 0 UNION ALL " +
+ "SELECT orderkey, day(receiptdate) FROM lineitem WHERE orderkey % 31 <> 0");
}
@Test
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableGroupedExecution.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableExecution.java
similarity index 81%
rename from presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableGroupedExecution.java
rename to presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableExecution.java
index 5d6e6ee703a55..07df5b2767d4d 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableGroupedExecution.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveRecoverableExecution.java
@@ -15,6 +15,7 @@
import com.facebook.airlift.log.Logger;
import com.facebook.presto.Session;
+import com.facebook.presto.metadata.AllNodes;
import com.facebook.presto.server.testing.TestingPrestoServer;
import com.facebook.presto.spi.security.Identity;
import com.facebook.presto.spi.security.SelectedRole;
@@ -25,6 +26,7 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
+import io.airlift.units.Duration;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -34,26 +36,31 @@
import java.util.List;
import java.util.Optional;
import java.util.OptionalLong;
+import java.util.concurrent.TimeoutException;
import static com.facebook.presto.SystemSessionProperties.COLOCATED_JOIN;
import static com.facebook.presto.SystemSessionProperties.CONCURRENT_LIFESPANS_PER_NODE;
import static com.facebook.presto.SystemSessionProperties.DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION;
+import static com.facebook.presto.SystemSessionProperties.EXCHANGE_MATERIALIZATION_STRATEGY;
import static com.facebook.presto.SystemSessionProperties.GROUPED_EXECUTION_FOR_AGGREGATION;
import static com.facebook.presto.SystemSessionProperties.GROUPED_EXECUTION_FOR_ELIGIBLE_TABLE_SCANS;
+import static com.facebook.presto.SystemSessionProperties.HASH_PARTITION_COUNT;
+import static com.facebook.presto.SystemSessionProperties.MAX_STAGE_RETRIES;
+import static com.facebook.presto.SystemSessionProperties.PARTITIONING_PROVIDER_CATALOG;
import static com.facebook.presto.SystemSessionProperties.RECOVERABLE_GROUPED_EXECUTION;
import static com.facebook.presto.SystemSessionProperties.REDISTRIBUTE_WRITES;
import static com.facebook.presto.SystemSessionProperties.SCALE_WRITERS;
import static com.facebook.presto.SystemSessionProperties.TASK_PARTITIONED_WRITER_COUNT;
import static com.facebook.presto.SystemSessionProperties.TASK_WRITER_COUNT;
-import static com.facebook.presto.execution.TaskState.RUNNING;
import static com.facebook.presto.hive.HiveQueryRunner.HIVE_CATALOG;
import static com.facebook.presto.hive.HiveQueryRunner.TPCH_BUCKETED_SCHEMA;
-import static com.facebook.presto.hive.HiveQueryRunner.createQueryRunner;
import static com.facebook.presto.hive.HiveSessionProperties.VIRTUAL_BUCKET_COUNT;
+import static com.facebook.presto.spi.security.SelectedRole.Type.ALL;
import static com.facebook.presto.spi.security.SelectedRole.Type.ROLE;
import static com.facebook.presto.testing.TestingSession.testSessionBuilder;
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
import static io.airlift.tpch.TpchTable.ORDERS;
+import static java.lang.String.format;
import static java.lang.Thread.sleep;
import static java.util.Collections.shuffle;
import static java.util.concurrent.Executors.newCachedThreadPool;
@@ -61,12 +68,11 @@
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toList;
import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertTrue;
@Test(singleThreaded = true)
-public class TestHiveRecoverableGroupedExecution
+public class TestHiveRecoverableExecution
{
- private static final Logger log = Logger.get(TestHiveRecoverableGroupedExecution.class);
+ private static final Logger log = Logger.get(TestHiveRecoverableExecution.class);
private static final int TEST_TIMEOUT = 120_000;
private static final int INVOCATION_COUNT = 1;
@@ -78,26 +84,42 @@ public class TestHiveRecoverableGroupedExecution
public void setUp()
throws Exception
{
- queryRunner = createQueryRunner(
+ queryRunner = createQueryRunner();
+ executor = listeningDecorator(newCachedThreadPool());
+ }
+
+ private DistributedQueryRunner createQueryRunner()
+ throws Exception
+ {
+ ImmutableMap.Builder extraPropertiesBuilder = ImmutableMap.builder()
+ // task get results timeout has to be significantly higher that the task status update timeout
+ .put("exchange.max-error-duration", "5m")
+ // set the timeout of a single HTTP request to 1s, to make sure that the task result requests are actually failing
+ // The default is 10s, that might not be sufficient to make sure that the request fails before the recoverable execution kicks in
+ .put("exchange.http-client.request-timeout", "1s");
+
+ ImmutableMap.Builder extraCoordinatorPropertiesBuilder = ImmutableMap.builder();
+
+ extraCoordinatorPropertiesBuilder
+ // decrease the heartbeat interval so we detect failed nodes faster
+ .put("failure-detector.heartbeat-interval", "1s")
+ .put("failure-detector.http-client.request-timeout", "500ms")
+ .put("failure-detector.exponential-decay-seconds", "1")
+ .put("failure-detector.threshold", "0.1")
+ // allow 2 out of 4 tasks to fail
+ .put("max-failed-task-percentage", "0.6")
+ // set the timeout of the task update requests to something low to improve overall test latency
+ .put("scheduler.http-client.request-timeout", "5s")
+ // this effectively disables the retries
+ .put("query.remote-task.max-error-duration", "1s");
+
+ return HiveQueryRunner.createQueryRunner(
ImmutableList.of(ORDERS),
// extra properties
- ImmutableMap.of(
- // task get results timeout has to be significantly higher that the task status update timeout
- "exchange.max-error-duration", "5m",
- // set the timeout of a single HTTP request to 1s, to make sure that the task result requests are actually failing
- // The default is 10s, that might not be sufficient to make sure that the request fails before the recoverable execution kicks in
- "exchange.http-client.request-timeout", "1s"),
+ extraPropertiesBuilder.build(),
// extra coordinator properties
- ImmutableMap.of(
- // set the timeout of the task update requests to something low to improve overall test latency
- "scheduler.http-client.request-timeout", "1s",
- // this effectively disables the retries
- "query.remote-task.max-error-duration", "1s",
- // allow 2 out of 4 tasks to fail
- "max-failed-task-percentage", "0.6"),
-
+ extraCoordinatorPropertiesBuilder.build(),
Optional.empty());
- executor = listeningDecorator(newCachedThreadPool());
}
@AfterClass(alwaysRun = true)
@@ -125,6 +147,7 @@ public void testCreateBucketedTable(int writerConcurrency)
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE create_bucketed_table_1\n" +
@@ -164,6 +187,7 @@ public void testInsertBucketedTable(int writerConcurrency)
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE insert_bucketed_table_1\n" +
@@ -207,6 +231,7 @@ public void testCreateUnbucketedTableWithGroupedExecution(int writerConcurrency)
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE create_unbucketed_table_with_grouped_execution_1\n" +
@@ -246,6 +271,7 @@ public void testInsertUnbucketedTableWithGroupedExecution(int writerConcurrency)
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE insert_unbucketed_table_with_grouped_execution_1\n" +
@@ -289,6 +315,7 @@ public void testScanFilterProjectionOnlyQueryOnUnbucketedTable(int writerConcurr
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE scan_filter_projection_only_query_on_unbucketed_table AS\n" +
@@ -311,6 +338,7 @@ public void testUnionAll(int writerConcurrency)
throws Exception
{
testRecoverableGroupedExecution(
+ queryRunner,
writerConcurrency,
ImmutableList.of(
"CREATE TABLE test_union_all AS\n" +
@@ -332,7 +360,30 @@ public void testUnionAll(int writerConcurrency)
"DROP TABLE IF EXISTS test_union_all_failure"));
}
+ @Test(invocationCount = INVOCATION_COUNT)
+ public void testCountOnUnbucketedTable()
+ throws Exception
+ {
+ testRecoverableGroupedExecution(
+ queryRunner,
+ 4,
+ ImmutableList.of(
+ "CREATE TABLE test_table AS\n" +
+ "SELECT orderkey, comment\n" +
+ "FROM orders\n"),
+ "CREATE TABLE test_success AS\n" +
+ "SELECT count(*) as a, comment FROM test_table group by comment",
+ "create table test_failure AS\n" +
+ "SELECT count(*) as a, comment FROM test_table group by comment",
+ 14995, // there are 14995 distinct comments in the orders table
+ ImmutableList.of(
+ "DROP TABLE IF EXISTS test_table",
+ "DROP TABLE IF EXISTS test_success",
+ "DROP TABLE IF EXISTS test_failure"));
+ }
+
private void testRecoverableGroupedExecution(
+ DistributedQueryRunner queryRunner,
int writerConcurrency,
List preQueries,
@Language("SQL") String queryWithoutFailure,
@@ -341,7 +392,12 @@ private void testRecoverableGroupedExecution(
List postQueries)
throws Exception
{
+ waitUntilAllNodesAreHealthy(queryRunner, new Duration(10, SECONDS));
+
Session recoverableSession = createRecoverableSession(writerConcurrency);
+ for (@Language("SQL") String postQuery : postQueries) {
+ queryRunner.execute(recoverableSession, postQuery);
+ }
try {
for (@Language("SQL") String preQuery : preQueries) {
queryRunner.execute(recoverableSession, preQuery);
@@ -371,24 +427,10 @@ private void testRecoverableGroupedExecution(
// kill worker2 only after the task has been scheduled
TestingPrestoServer worker2 = workers.get(1);
- Stopwatch stopwatch = Stopwatch.createStarted();
- while (true) {
- // wait for a while
- sleep(500);
-
- // if the task is already running - move ahead
- if (hasTaskRunning(worker2)) {
- break;
- }
-
- // don't fail the test if task execution already finished
- if (stopwatch.elapsed(SECONDS) > 5) {
- break;
- }
- }
+ sleep(1000);
worker2.stopResponding();
- assertEquals(result.get(60, SECONDS).getUpdateCount(), OptionalLong.of(expectedUpdateCount));
+ assertEquals(result.get(1000, SECONDS).getUpdateCount(), OptionalLong.of(expectedUpdateCount));
log.info("Query with recovery took %sms", recoveryStopwatch.elapsed(MILLISECONDS));
}
finally {
@@ -401,27 +443,34 @@ private void testRecoverableGroupedExecution(
}
}
+ private static void waitUntilAllNodesAreHealthy(DistributedQueryRunner queryRunner, Duration timeout)
+ throws TimeoutException, InterruptedException
+ {
+ TestingPrestoServer coordinator = queryRunner.getCoordinator();
+ long deadline = System.currentTimeMillis() + timeout.toMillis();
+ while (System.currentTimeMillis() < deadline) {
+ AllNodes allNodes = coordinator.refreshNodes();
+ if (allNodes.getActiveNodes().size() == queryRunner.getNodeCount()) {
+ return;
+ }
+ sleep(1000);
+ }
+ throw new TimeoutException(format("one of the nodes is still missing after: %s", timeout));
+ }
+
private static void cancelAllQueries(DistributedQueryRunner queryRunner)
{
queryRunner.getQueries().forEach(query -> queryRunner.getCoordinator().getQueryManager().cancelQuery(query.getQueryId()));
- queryRunner.getQueries().forEach(query -> assertTrue(query.getState().isDone()));
}
private static void cancelAllTasks(DistributedQueryRunner queryRunner)
{
- queryRunner.getServers().forEach(TestHiveRecoverableGroupedExecution::cancelAllTasks);
+ queryRunner.getServers().forEach(TestHiveRecoverableExecution::cancelAllTasks);
}
private static void cancelAllTasks(TestingPrestoServer server)
{
server.getTaskManager().getAllTaskInfo().forEach(task -> server.getTaskManager().cancelTask(task.getTaskStatus().getTaskId()));
- server.getTaskManager().getAllTaskInfo().forEach(task -> assertTrue(task.getTaskStatus().getState().isDone()));
- }
-
- private static boolean hasTaskRunning(TestingPrestoServer server)
- {
- return server.getTaskManager().getAllTaskInfo().stream()
- .anyMatch(taskInfo -> taskInfo.getTaskStatus().getState() == RUNNING);
}
private static Session createRecoverableSession(int writerConcurrency)
@@ -446,6 +495,10 @@ private static Session createRecoverableSession(int writerConcurrency)
.setSystemProperty(GROUPED_EXECUTION_FOR_ELIGIBLE_TABLE_SCANS, "true")
.setSystemProperty(TASK_WRITER_COUNT, Integer.toString(writerConcurrency))
.setSystemProperty(TASK_PARTITIONED_WRITER_COUNT, Integer.toString(writerConcurrency))
+ .setSystemProperty(PARTITIONING_PROVIDER_CATALOG, "hive")
+ .setSystemProperty(EXCHANGE_MATERIALIZATION_STRATEGY, ALL.name())
+ .setSystemProperty(HASH_PARTITION_COUNT, "11")
+ .setSystemProperty(MAX_STAGE_RETRIES, "4")
.setCatalogSessionProperty(HIVE_CATALOG, VIRTUAL_BUCKET_COUNT, "16")
.setCatalog(HIVE_CATALOG)
.setSchema(TPCH_BUCKETED_SCHEMA)
diff --git a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/JmxQueryRunner.java b/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/JmxQueryRunner.java
index 7c5d9f6009fed..8038a05849c2e 100644
--- a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/JmxQueryRunner.java
+++ b/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/JmxQueryRunner.java
@@ -15,6 +15,7 @@
import com.facebook.presto.Session;
import com.facebook.presto.tests.DistributedQueryRunner;
+import com.google.common.collect.ImmutableMap;
import static com.facebook.airlift.testing.Closeables.closeAllSuppress;
import static com.facebook.presto.connector.jmx.JmxMetadata.JMX_SCHEMA_NAME;
@@ -30,7 +31,11 @@ public static DistributedQueryRunner createJmxQueryRunner()
{
DistributedQueryRunner queryRunner = null;
try {
- queryRunner = new DistributedQueryRunner(createSession(), 3);
+ queryRunner = DistributedQueryRunner.builder(createSession())
+ .setNodeCount(3)
+ // disable failure-detector to prevent flaky tests since the jmx tests rely on the number of nodes being consistent
+ .setCoordinatorProperties(ImmutableMap.of("failure-detector.enabled", "false"))
+ .build();
queryRunner.installPlugin(new JmxPlugin());
queryRunner.createCatalog("jmx", "jmx");
diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java
index 1dbb556f38109..f9652a952b9c9 100644
--- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java
+++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java
@@ -70,6 +70,7 @@ public final class SystemSessionProperties
public static final String DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION = "dynamic_schedule_for_grouped_execution";
public static final String RECOVERABLE_GROUPED_EXECUTION = "recoverable_grouped_execution";
public static final String MAX_FAILED_TASK_PERCENTAGE = "max_failed_task_percentage";
+ public static final String MAX_STAGE_RETRIES = "max_stage_retries";
public static final String PREFER_STREAMING_OPERATORS = "prefer_streaming_operators";
public static final String TASK_WRITER_COUNT = "task_writer_count";
public static final String TASK_PARTITIONED_WRITER_COUNT = "task_partitioned_writer_count";
@@ -142,6 +143,7 @@ public final class SystemSessionProperties
public static final String LIST_BUILT_IN_FUNCTIONS_ONLY = "list_built_in_functions_only";
public static final String PARTITIONING_PRECISION_STRATEGY = "partitioning_precision_strategy";
public static final String EXPERIMENTAL_FUNCTIONS_ENABLED = "experimental_functions_enabled";
+ public static final String USE_LEGACY_SCHEDULER = "use_legacy_scheduler";
private final List> sessionProperties;
@@ -246,6 +248,11 @@ public SystemSessionProperties(
"Experimental: Use recoverable grouped execution when possible",
featuresConfig.isRecoverableGroupedExecutionEnabled(),
false),
+ integerProperty(
+ MAX_STAGE_RETRIES,
+ "Maximum number of times that stages can be retried",
+ featuresConfig.getMaxStageRetries(),
+ false),
booleanProperty(
PREFER_STREAMING_OPERATORS,
"Prefer source table layouts that produce streaming operators",
@@ -710,6 +717,12 @@ public SystemSessionProperties(
EXPERIMENTAL_FUNCTIONS_ENABLED,
"Enable listing of functions marked as experimental",
featuresConfig.isExperimentalFunctionsEnabled(),
+ false),
+
+ booleanProperty(
+ USE_LEGACY_SCHEDULER,
+ "Use version of scheduler before refactorings for section retries",
+ featuresConfig.isUseLegacyScheduler(),
false));
}
@@ -792,6 +805,11 @@ public static double getMaxFailedTaskPercentage(Session session)
return session.getSystemProperty(MAX_FAILED_TASK_PERCENTAGE, Double.class);
}
+ public static int getMaxStageRetries(Session session)
+ {
+ return session.getSystemProperty(MAX_STAGE_RETRIES, Integer.class);
+ }
+
public static boolean preferStreamingOperators(Session session)
{
return session.getSystemProperty(PREFER_STREAMING_OPERATORS, Boolean.class);
@@ -1206,4 +1224,9 @@ public static boolean isExperimentalFunctionsEnabled(Session session)
{
return session.getSystemProperty(EXPERIMENTAL_FUNCTIONS_ENABLED, Boolean.class);
}
+
+ public static boolean isUseLegacyScheduler(Session session)
+ {
+ return session.getSystemProperty(USE_LEGACY_SCHEDULER, Boolean.class);
+ }
}
diff --git a/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java b/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java
index d38b4949fea64..f7f3a2b6c1137 100644
--- a/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java
+++ b/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java
@@ -151,6 +151,7 @@ public void queryImmediateFailureEvent(BasicQueryInfo queryInfo, ExecutionFailur
ofMillis(0),
ofMillis(0),
ofMillis(0),
+ ofMillis(0),
Optional.empty(),
0,
0,
@@ -256,6 +257,7 @@ private QueryStatistics createQueryStatistics(QueryInfo queryInfo)
return new QueryStatistics(
ofMillis(queryStats.getTotalCpuTime().toMillis()),
+ ofMillis(queryStats.getRetriedCpuTime().toMillis()),
ofMillis(queryStats.getTotalScheduledTime().toMillis()),
ofMillis(queryStats.getQueuedTime().toMillis()),
Optional.of(ofMillis(queryStats.getAnalysisTime().toMillis())),
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java
index e6cfde0bec4e6..fb7c1f5c8fc10 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java
@@ -465,6 +465,7 @@ private QueryStats getQueryStats(Optional rootStage)
long totalScheduledTime = 0;
long totalCpuTime = 0;
+ long retriedCpuTime = 0;
long totalBlockedTime = 0;
long rawInputDataSize = 0;
@@ -506,6 +507,7 @@ private QueryStats getQueryStats(Optional rootStage)
totalMemoryReservation += stageExecutionStats.getTotalMemoryReservation().toBytes();
totalScheduledTime += stageExecutionStats.getTotalScheduledTime().roundTo(MILLISECONDS);
totalCpuTime += stageExecutionStats.getTotalCpuTime().roundTo(MILLISECONDS);
+ retriedCpuTime += computeRetriedCpuTime(stageInfo);
totalBlockedTime += stageExecutionStats.getTotalBlockedTime().roundTo(MILLISECONDS);
if (!stageInfo.getLatestAttemptExecutionInfo().getState().isDone()) {
fullyBlocked &= stageExecutionStats.isFullyBlocked();
@@ -589,6 +591,7 @@ private QueryStats getQueryStats(Optional rootStage)
succinctDuration(totalScheduledTime, MILLISECONDS),
succinctDuration(totalCpuTime, MILLISECONDS),
+ succinctDuration(retriedCpuTime, MILLISECONDS),
succinctDuration(totalBlockedTime, MILLISECONDS),
fullyBlocked,
blockedReasons,
@@ -611,6 +614,15 @@ private QueryStats getQueryStats(Optional rootStage)
operatorStatsSummary.build());
}
+ private static long computeRetriedCpuTime(StageInfo stageInfo)
+ {
+ long stageRetriedCpuTime = stageInfo.getPreviousAttemptsExecutionInfos().stream()
+ .mapToLong(executionInfo -> executionInfo.getStats().getTotalCpuTime().roundTo(MILLISECONDS))
+ .sum();
+ long taskRetriedCpuTime = stageInfo.getLatestAttemptExecutionInfo().getStats().getRetriedCpuTime().roundTo(MILLISECONDS);
+ return stageRetriedCpuTime + taskRetriedCpuTime;
+ }
+
public VersionedMemoryPoolId getMemoryPool()
{
return memoryPool.get();
@@ -1034,7 +1046,6 @@ public void pruneQueryInfo()
private static StageExecutionInfo pruneStageExecutionInfo(StageExecutionInfo info)
{
return new StageExecutionInfo(
- info.getStageExecutionId(),
info.getState(),
info.getStats(),
// Remove the tasks
@@ -1075,6 +1086,7 @@ private static QueryStats pruneQueryStats(QueryStats queryStats)
queryStats.isScheduled(),
queryStats.getTotalScheduledTime(),
queryStats.getTotalCpuTime(),
+ queryStats.getRetriedCpuTime(),
queryStats.getTotalBlockedTime(),
queryStats.isFullyBlocked(),
queryStats.getBlockedReasons(),
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/QueryStats.java b/presto-main/src/main/java/com/facebook/presto/execution/QueryStats.java
index b869de1d56e66..e9e245419d22e 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/QueryStats.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/QueryStats.java
@@ -75,6 +75,7 @@ public class QueryStats
private final boolean scheduled;
private final Duration totalScheduledTime;
private final Duration totalCpuTime;
+ private final Duration retriedCpuTime;
private final Duration totalBlockedTime;
private final boolean fullyBlocked;
private final Set blockedReasons;
@@ -135,6 +136,7 @@ public QueryStats(
@JsonProperty("scheduled") boolean scheduled,
@JsonProperty("totalScheduledTime") Duration totalScheduledTime,
@JsonProperty("totalCpuTime") Duration totalCpuTime,
+ @JsonProperty("retriedCpuTime") Duration retriedCpuTime,
@JsonProperty("totalBlockedTime") Duration totalBlockedTime,
@JsonProperty("fullyBlocked") boolean fullyBlocked,
@JsonProperty("blockedReasons") Set blockedReasons,
@@ -201,6 +203,7 @@ public QueryStats(
this.scheduled = scheduled;
this.totalScheduledTime = requireNonNull(totalScheduledTime, "totalScheduledTime is null");
this.totalCpuTime = requireNonNull(totalCpuTime, "totalCpuTime is null");
+ this.retriedCpuTime = requireNonNull(retriedCpuTime, "totalCpuTime is null");
this.totalBlockedTime = requireNonNull(totalBlockedTime, "totalBlockedTime is null");
this.fullyBlocked = fullyBlocked;
this.blockedReasons = ImmutableSet.copyOf(requireNonNull(blockedReasons, "blockedReasons is null"));
@@ -263,6 +266,7 @@ public static QueryStats immediateFailureQueryStats()
new Duration(0, MILLISECONDS),
new Duration(0, MILLISECONDS),
new Duration(0, MILLISECONDS),
+ new Duration(0, MILLISECONDS),
false,
ImmutableSet.of(),
new DataSize(0, BYTE),
@@ -460,6 +464,12 @@ public Duration getTotalCpuTime()
return totalCpuTime;
}
+ @JsonProperty
+ public Duration getRetriedCpuTime()
+ {
+ return retriedCpuTime;
+ }
+
@JsonProperty
public Duration getTotalBlockedTime()
{
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java
index afaa5e442b996..462d2acb855cd 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java
@@ -23,14 +23,15 @@
import com.facebook.presto.execution.buffer.OutputBuffers;
import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
import com.facebook.presto.execution.scheduler.ExecutionPolicy;
-import com.facebook.presto.execution.scheduler.NodeScheduler;
+import com.facebook.presto.execution.scheduler.LegacySqlQueryScheduler;
+import com.facebook.presto.execution.scheduler.SectionExecutionFactory;
import com.facebook.presto.execution.scheduler.SplitSchedulerStats;
import com.facebook.presto.execution.scheduler.SqlQueryScheduler;
+import com.facebook.presto.execution.scheduler.SqlQuerySchedulerInterface;
import com.facebook.presto.execution.warnings.WarningCollector;
-import com.facebook.presto.failureDetector.FailureDetector;
import com.facebook.presto.memory.VersionedMemoryPoolId;
+import com.facebook.presto.metadata.InternalNodeManager;
import com.facebook.presto.metadata.Metadata;
-import com.facebook.presto.operator.ForScheduler;
import com.facebook.presto.security.AccessControl;
import com.facebook.presto.server.BasicQueryInfo;
import com.facebook.presto.spi.ConnectorId;
@@ -49,7 +50,6 @@
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.planner.InputExtractor;
import com.facebook.presto.sql.planner.LogicalPlanner;
-import com.facebook.presto.sql.planner.NodePartitioningManager;
import com.facebook.presto.sql.planner.OutputExtractor;
import com.facebook.presto.sql.planner.PartitioningHandle;
import com.facebook.presto.sql.planner.Plan;
@@ -76,15 +76,14 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static com.facebook.airlift.concurrent.MoreFutures.addExceptionCallback;
import static com.facebook.airlift.concurrent.MoreFutures.addSuccessCallback;
+import static com.facebook.presto.SystemSessionProperties.isUseLegacyScheduler;
import static com.facebook.presto.execution.buffer.OutputBuffers.BROADCAST_PARTITION_ID;
import static com.facebook.presto.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers;
-import static com.facebook.presto.execution.scheduler.SqlQueryScheduler.createSqlQueryScheduler;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Throwables.throwIfInstanceOf;
@@ -105,20 +104,16 @@ public class SqlQueryExecution
private final Metadata metadata;
private final SqlParser sqlParser;
private final SplitManager splitManager;
- private final NodePartitioningManager nodePartitioningManager;
- private final NodeScheduler nodeScheduler;
private final List planOptimizers;
private final PlanFragmenter planFragmenter;
private final RemoteTaskFactory remoteTaskFactory;
private final LocationFactory locationFactory;
- private final int scheduleSplitBatchSize;
private final ExecutorService queryExecutor;
- private final ScheduledExecutorService schedulerExecutor;
- private final FailureDetector failureDetector;
+ private final SectionExecutionFactory sectionExecutionFactory;
+ private final InternalNodeManager internalNodeManager;
- private final AtomicReference queryScheduler = new AtomicReference<>();
+ private final AtomicReference queryScheduler = new AtomicReference<>();
private final AtomicReference queryPlan = new AtomicReference<>();
- private final NodeTaskMap nodeTaskMap;
private final ExecutionPolicy executionPolicy;
private final SplitSchedulerStats schedulerStats;
private final Analysis analysis;
@@ -138,17 +133,13 @@ private SqlQueryExecution(
AccessControl accessControl,
SqlParser sqlParser,
SplitManager splitManager,
- NodePartitioningManager nodePartitioningManager,
- NodeScheduler nodeScheduler,
List planOptimizers,
PlanFragmenter planFragmenter,
RemoteTaskFactory remoteTaskFactory,
LocationFactory locationFactory,
- int scheduleSplitBatchSize,
ExecutorService queryExecutor,
- ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- NodeTaskMap nodeTaskMap,
+ SectionExecutionFactory sectionExecutionFactory,
+ InternalNodeManager internalNodeManager,
QueryExplainer queryExplainer,
ExecutionPolicy executionPolicy,
SplitSchedulerStats schedulerStats,
@@ -161,23 +152,17 @@ private SqlQueryExecution(
this.metadata = requireNonNull(metadata, "metadata is null");
this.sqlParser = requireNonNull(sqlParser, "sqlParser is null");
this.splitManager = requireNonNull(splitManager, "splitManager is null");
- this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null");
- this.nodeScheduler = requireNonNull(nodeScheduler, "nodeScheduler is null");
this.planOptimizers = requireNonNull(planOptimizers, "planOptimizers is null");
this.planFragmenter = requireNonNull(planFragmenter, "planFragmenter is null");
this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
this.queryExecutor = requireNonNull(queryExecutor, "queryExecutor is null");
- this.schedulerExecutor = requireNonNull(schedulerExecutor, "schedulerExecutor is null");
- this.failureDetector = requireNonNull(failureDetector, "failureDetector is null");
- this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null");
+ this.sectionExecutionFactory = requireNonNull(sectionExecutionFactory, "sectionExecutionFactory is null");
+ this.internalNodeManager = requireNonNull(internalNodeManager, "internalNodeManager is null");
this.executionPolicy = requireNonNull(executionPolicy, "executionPolicy is null");
this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null");
this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null");
this.costCalculator = requireNonNull(costCalculator, "costCalculator is null");
- checkArgument(scheduleSplitBatchSize > 0, "scheduleSplitBatchSize must be greater than 0");
- this.scheduleSplitBatchSize = scheduleSplitBatchSize;
-
requireNonNull(query, "query is null");
requireNonNull(session, "session is null");
requireNonNull(self, "self is null");
@@ -216,14 +201,14 @@ private SqlQueryExecution(
stateMachine.setUpdateType(analysis.getUpdateType());
// when the query finishes cache the final query info, and clear the reference to the output stage
- AtomicReference queryScheduler = this.queryScheduler;
+ AtomicReference queryScheduler = this.queryScheduler;
stateMachine.addStateChangeListener(state -> {
if (!state.isDone()) {
return;
}
// query is now done, so abort any work that is still running
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
if (scheduler != null) {
scheduler.abort();
}
@@ -251,7 +236,7 @@ public DataSize getUserMemoryReservation()
// acquire reference to scheduler before checking finalQueryInfo, because
// state change listener sets finalQueryInfo and then clears scheduler when
// the query finishes.
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
Optional finalQueryInfo = stateMachine.getFinalQueryInfo();
if (finalQueryInfo.isPresent()) {
return finalQueryInfo.get().getQueryStats().getUserMemoryReservation();
@@ -268,7 +253,7 @@ public DataSize getTotalMemoryReservation()
// acquire reference to scheduler before checking finalQueryInfo, because
// state change listener sets finalQueryInfo and then clears scheduler when
// the query finishes.
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
Optional finalQueryInfo = stateMachine.getFinalQueryInfo();
if (finalQueryInfo.isPresent()) {
return finalQueryInfo.get().getQueryStats().getTotalMemoryReservation();
@@ -306,7 +291,7 @@ public Optional getEndTime()
@Override
public Duration getTotalCpuTime()
{
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
Optional finalQueryInfo = stateMachine.getFinalQueryInfo();
if (finalQueryInfo.isPresent()) {
return finalQueryInfo.get().getQueryStats().getTotalCpuTime();
@@ -322,7 +307,7 @@ public BasicQueryInfo getBasicQueryInfo()
{
return stateMachine.getFinalQueryInfo()
.map(BasicQueryInfo::new)
- .orElseGet(() -> stateMachine.getBasicQueryInfo(Optional.ofNullable(queryScheduler.get()).map(SqlQueryScheduler::getBasicStageStats)));
+ .orElseGet(() -> stateMachine.getBasicQueryInfo(Optional.ofNullable(queryScheduler.get()).map(SqlQuerySchedulerInterface::getBasicStageStats)));
}
@Override
@@ -371,7 +356,7 @@ private void startExecution()
}
// if query is not finished, start the scheduler, otherwise cancel it
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
if (!stateMachine.isDone()) {
scheduler.start();
@@ -493,25 +478,33 @@ private void planDistribution(PlanRoot plan)
SplitSourceFactory splitSourceFactory = new SplitSourceFactory(splitSourceProvider);
// build the stage execution objects (this doesn't schedule execution)
- SqlQueryScheduler scheduler = createSqlQueryScheduler(
- stateMachine,
- locationFactory,
- outputStagePlan,
- nodePartitioningManager,
- nodeScheduler,
- remoteTaskFactory,
- splitSourceFactory,
- stateMachine.getSession(),
- plan.isSummarizeTaskInfos(),
- scheduleSplitBatchSize,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- rootOutputBuffers,
- nodeTaskMap,
- executionPolicy,
- schedulerStats,
- metadata);
+ SqlQuerySchedulerInterface scheduler = isUseLegacyScheduler(getSession()) ?
+ LegacySqlQueryScheduler.createSqlQueryScheduler(
+ locationFactory,
+ executionPolicy,
+ queryExecutor,
+ schedulerStats,
+ sectionExecutionFactory,
+ remoteTaskFactory,
+ splitSourceFactory,
+ stateMachine.getSession(),
+ stateMachine,
+ outputStagePlan,
+ rootOutputBuffers,
+ plan.isSummarizeTaskInfos()) :
+ SqlQueryScheduler.createSqlQueryScheduler(
+ locationFactory,
+ executionPolicy,
+ queryExecutor,
+ schedulerStats,
+ sectionExecutionFactory,
+ remoteTaskFactory,
+ splitSourceFactory,
+ internalNodeManager,
+ stateMachine.getSession(),
+ stateMachine,
+ outputStagePlan,
+ plan.isSummarizeTaskInfos());
queryScheduler.set(scheduler);
@@ -535,7 +528,7 @@ public void cancelStage(StageId stageId)
requireNonNull(stageId, "stageId is null");
try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId())) {
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
if (scheduler != null) {
scheduler.cancelStage(stageId);
}
@@ -593,7 +586,7 @@ public QueryInfo getQueryInfo()
// acquire reference to scheduler before checking finalQueryInfo, because
// state change listener sets finalQueryInfo and then clears scheduler when
// the query finishes.
- SqlQueryScheduler scheduler = queryScheduler.get();
+ SqlQuerySchedulerInterface scheduler = queryScheduler.get();
return stateMachine.getFinalQueryInfo().orElseGet(() -> buildQueryInfo(scheduler));
}
@@ -611,7 +604,7 @@ public Plan getQueryPlan()
return queryPlan.get();
}
- private QueryInfo buildQueryInfo(SqlQueryScheduler scheduler)
+ private QueryInfo buildQueryInfo(SqlQuerySchedulerInterface scheduler)
{
Optional stageInfo = Optional.empty();
if (scheduler != null) {
@@ -660,13 +653,10 @@ public static class SqlQueryExecutionFactory
implements QueryExecutionFactory
{
private final SplitSchedulerStats schedulerStats;
- private final int scheduleSplitBatchSize;
private final Metadata metadata;
private final AccessControl accessControl;
private final SqlParser sqlParser;
private final SplitManager splitManager;
- private final NodePartitioningManager nodePartitioningManager;
- private final NodeScheduler nodeScheduler;
private final List planOptimizers;
private final PlanFragmenter planFragmenter;
private final RemoteTaskFactory remoteTaskFactory;
@@ -674,9 +664,8 @@ public static class SqlQueryExecutionFactory
private final QueryExplainer queryExplainer;
private final LocationFactory locationFactory;
private final ExecutorService queryExecutor;
- private final ScheduledExecutorService schedulerExecutor;
- private final FailureDetector failureDetector;
- private final NodeTaskMap nodeTaskMap;
+ private final SectionExecutionFactory sectionExecutionFactory;
+ private final InternalNodeManager internalNodeManager;
private final Map executionPolicies;
private final ClusterSizeMonitor clusterSizeMonitor;
private final StatsCalculator statsCalculator;
@@ -689,16 +678,13 @@ public static class SqlQueryExecutionFactory
SqlParser sqlParser,
LocationFactory locationFactory,
SplitManager splitManager,
- NodePartitioningManager nodePartitioningManager,
- NodeScheduler nodeScheduler,
PlanOptimizers planOptimizers,
PlanFragmenter planFragmenter,
RemoteTaskFactory remoteTaskFactory,
TransactionManager transactionManager,
@ForQueryExecution ExecutorService queryExecutor,
- @ForScheduler ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- NodeTaskMap nodeTaskMap,
+ SectionExecutionFactory sectionExecutionFactory,
+ InternalNodeManager internalNodeManager,
QueryExplainer queryExplainer,
Map executionPolicies,
SplitSchedulerStats schedulerStats,
@@ -708,22 +694,18 @@ public static class SqlQueryExecutionFactory
{
requireNonNull(config, "config is null");
this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null");
- this.scheduleSplitBatchSize = config.getScheduleSplitBatchSize();
this.metadata = requireNonNull(metadata, "metadata is null");
this.accessControl = requireNonNull(accessControl, "accessControl is null");
this.sqlParser = requireNonNull(sqlParser, "sqlParser is null");
this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
this.splitManager = requireNonNull(splitManager, "splitManager is null");
- this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null");
- this.nodeScheduler = requireNonNull(nodeScheduler, "nodeScheduler is null");
requireNonNull(planOptimizers, "planOptimizers is null");
this.planFragmenter = requireNonNull(planFragmenter, "planFragmenter is null");
this.remoteTaskFactory = requireNonNull(remoteTaskFactory, "remoteTaskFactory is null");
this.transactionManager = requireNonNull(transactionManager, "transactionManager is null");
this.queryExecutor = requireNonNull(queryExecutor, "queryExecutor is null");
- this.schedulerExecutor = requireNonNull(schedulerExecutor, "schedulerExecutor is null");
- this.failureDetector = requireNonNull(failureDetector, "failureDetector is null");
- this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null");
+ this.sectionExecutionFactory = requireNonNull(sectionExecutionFactory, "sectionExecutionFactory is null");
+ this.internalNodeManager = requireNonNull(internalNodeManager, "internalNodeManager is null");
this.queryExplainer = requireNonNull(queryExplainer, "queryExplainer is null");
this.executionPolicies = requireNonNull(executionPolicies, "schedulerPolicies is null");
this.clusterSizeMonitor = requireNonNull(clusterSizeMonitor, "clusterSizeMonitor is null");
@@ -758,17 +740,13 @@ public QueryExecution createQueryExecution(
accessControl,
sqlParser,
splitManager,
- nodePartitioningManager,
- nodeScheduler,
planOptimizers,
planFragmenter,
remoteTaskFactory,
locationFactory,
- scheduleSplitBatchSize,
queryExecutor,
- schedulerExecutor,
- failureDetector,
- nodeTaskMap,
+ sectionExecutionFactory,
+ internalNodeManager,
queryExplainer,
executionPolicy,
schedulerStats,
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java
index a6e7e12b99af2..aceef30a8bf23 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java
@@ -81,7 +81,7 @@
@ThreadSafe
public final class SqlStageExecution
{
- private static final Set RECOVERABLE_ERROR_CODES = ImmutableSet.of(
+ public static final Set RECOVERABLE_ERROR_CODES = ImmutableSet.of(
TOO_MANY_REQUESTS_FAILED.toErrorCode(),
PAGE_TRANSPORT_ERROR.toErrorCode(),
PAGE_TRANSPORT_TIMEOUT.toErrorCode(),
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionInfo.java b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionInfo.java
index 866ac13d94d08..43f459865ba69 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionInfo.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionInfo.java
@@ -24,7 +24,6 @@
public class StageExecutionInfo
{
- private final StageExecutionId stageExecutionId;
private final StageExecutionState state;
private final StageExecutionStats stats;
private final List tasks;
@@ -32,25 +31,17 @@ public class StageExecutionInfo
@JsonCreator
public StageExecutionInfo(
- @JsonProperty("stageExecutionId") StageExecutionId stageExecutionId,
@JsonProperty("state") StageExecutionState state,
@JsonProperty("stats") StageExecutionStats stats,
@JsonProperty("tasks") List tasks,
@JsonProperty("failureCause") Optional failureCause)
{
- this.stageExecutionId = requireNonNull(stageExecutionId, "stageExecutionId is null");
this.state = requireNonNull(state, "state is null");
this.stats = requireNonNull(stats, "stats is null");
this.tasks = ImmutableList.copyOf(requireNonNull(tasks, "tasks is null"));
this.failureCause = requireNonNull(failureCause, "failureCause is null");
}
- @JsonProperty
- public StageExecutionId getStageExecutionId()
- {
- return stageExecutionId;
- }
-
@JsonProperty
public StageExecutionState getState()
{
@@ -79,4 +70,13 @@ public boolean isFinal()
{
return state.isDone() && tasks.stream().allMatch(taskInfo -> taskInfo.getTaskStatus().getState().isDone());
}
+
+ public static StageExecutionInfo unscheduledExecutionInfo(int stageId, boolean isQueryDone)
+ {
+ return new StageExecutionInfo(
+ isQueryDone ? StageExecutionState.ABORTED : StageExecutionState.PLANNED,
+ StageExecutionStats.zero(stageId),
+ ImmutableList.of(),
+ Optional.empty());
+ }
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStateMachine.java
index 0b0f166841b1c..84394649f04ce 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStateMachine.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStateMachine.java
@@ -346,6 +346,7 @@ public StageExecutionInfo getStageExecutionInfo(Supplier> tas
long totalScheduledTime = 0;
long totalCpuTime = 0;
+ long retriedCpuTime = 0;
long totalBlockedTime = 0;
long rawInputDataSize = 0;
@@ -396,6 +397,9 @@ public StageExecutionInfo getStageExecutionInfo(Supplier> tas
totalScheduledTime += taskStats.getTotalScheduledTime().roundTo(NANOSECONDS);
totalCpuTime += taskStats.getTotalCpuTime().roundTo(NANOSECONDS);
+ if (state == FINISHED && taskInfo.getTaskStatus().getState() == TaskState.FAILED) {
+ retriedCpuTime += taskStats.getTotalCpuTime().roundTo(NANOSECONDS);
+ }
totalBlockedTime += taskStats.getTotalBlockedTime().roundTo(NANOSECONDS);
if (!taskState.isDone()) {
fullyBlocked &= taskStats.isFullyBlocked();
@@ -453,6 +457,7 @@ public StageExecutionInfo getStageExecutionInfo(Supplier> tas
succinctBytes(peakUserMemoryReservation),
succinctDuration(totalScheduledTime, NANOSECONDS),
succinctDuration(totalCpuTime, NANOSECONDS),
+ succinctDuration(retriedCpuTime, NANOSECONDS),
succinctDuration(totalBlockedTime, NANOSECONDS),
fullyBlocked && runningTasks > 0,
blockedReasons,
@@ -483,7 +488,6 @@ public StageExecutionInfo getStageExecutionInfo(Supplier> tas
failureInfo = Optional.of(failureCause.get());
}
return new StageExecutionInfo(
- stageExecutionId,
state,
stageExecutionStats,
taskInfos,
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStats.java b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStats.java
index bfd532748ed6c..3dcf6b5a8dd87 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStats.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionStats.java
@@ -13,6 +13,7 @@
*/
package com.facebook.presto.execution;
+import com.facebook.airlift.stats.Distribution;
import com.facebook.airlift.stats.Distribution.DistributionSnapshot;
import com.facebook.presto.operator.BlockedReason;
import com.facebook.presto.operator.OperatorStats;
@@ -33,8 +34,10 @@
import static com.facebook.presto.execution.StageExecutionState.RUNNING;
import static com.google.common.base.Preconditions.checkArgument;
+import static io.airlift.units.DataSize.Unit.BYTE;
import static java.lang.Math.min;
import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
@Immutable
public class StageExecutionStats
@@ -63,6 +66,7 @@ public class StageExecutionStats
private final Duration totalScheduledTime;
private final Duration totalCpuTime;
+ private final Duration retriedCpuTime;
private final Duration totalBlockedTime;
private final boolean fullyBlocked;
private final Set blockedReasons;
@@ -109,6 +113,7 @@ public StageExecutionStats(
@JsonProperty("totalScheduledTime") Duration totalScheduledTime,
@JsonProperty("totalCpuTime") Duration totalCpuTime,
+ @JsonProperty("retriedCpuTime") Duration retriedCpuTime,
@JsonProperty("totalBlockedTime") Duration totalBlockedTime,
@JsonProperty("fullyBlocked") boolean fullyBlocked,
@JsonProperty("blockedReasons") Set blockedReasons,
@@ -162,6 +167,7 @@ public StageExecutionStats(
this.totalScheduledTime = requireNonNull(totalScheduledTime, "totalScheduledTime is null");
this.totalCpuTime = requireNonNull(totalCpuTime, "totalCpuTime is null");
+ this.retriedCpuTime = requireNonNull(retriedCpuTime, "retriedCpuTime is null");
this.totalBlockedTime = requireNonNull(totalBlockedTime, "totalBlockedTime is null");
this.fullyBlocked = fullyBlocked;
this.blockedReasons = ImmutableSet.copyOf(requireNonNull(blockedReasons, "blockedReasons is null"));
@@ -294,6 +300,12 @@ public Duration getTotalCpuTime()
return totalCpuTime;
}
+ @JsonProperty
+ public Duration getRetriedCpuTime()
+ {
+ return retriedCpuTime;
+ }
+
@JsonProperty
public Duration getTotalBlockedTime()
{
@@ -398,4 +410,41 @@ public BasicStageExecutionStats toBasicStageStats(StageExecutionState stageExecu
blockedReasons,
progressPercentage);
}
+
+ public static StageExecutionStats zero(int stageId)
+ {
+ return new StageExecutionStats(
+ null,
+ new Distribution().snapshot(),
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ 0,
+ new DataSize(0, BYTE),
+ new DataSize(0, BYTE),
+ new DataSize(0, BYTE),
+ new Duration(0, NANOSECONDS),
+ new Duration(0, NANOSECONDS),
+ new Duration(0, NANOSECONDS),
+ new Duration(0, NANOSECONDS),
+ false,
+ ImmutableSet.of(),
+ new DataSize(0, BYTE),
+ 0,
+ new DataSize(0, BYTE),
+ 0,
+ new DataSize(0, BYTE),
+ new DataSize(0, BYTE),
+ 0,
+ new DataSize(0, BYTE),
+ new StageGcStatistics(stageId, 0, 0, 0, 0, 0, 0, 0),
+ ImmutableList.of());
+ }
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionPolicy.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionPolicy.java
index b9c1bc96ec79d..d443edf8c299c 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionPolicy.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionPolicy.java
@@ -13,15 +13,13 @@
*/
package com.facebook.presto.execution.scheduler;
-import com.facebook.presto.execution.SqlStageExecution;
-
import java.util.Collection;
public class AllAtOnceExecutionPolicy
implements ExecutionPolicy
{
@Override
- public ExecutionSchedule createExecutionSchedule(Collection stages)
+ public ExecutionSchedule createExecutionSchedule(Collection stages)
{
return new AllAtOnceExecutionSchedule(stages);
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionSchedule.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionSchedule.java
index 9d898f688beeb..60948e6a10c49 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionSchedule.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/AllAtOnceExecutionSchedule.java
@@ -50,26 +50,27 @@
public class AllAtOnceExecutionSchedule
implements ExecutionSchedule
{
- private final Set schedulingStages;
+ private final Set schedulingStages;
- public AllAtOnceExecutionSchedule(Collection stages)
+ public AllAtOnceExecutionSchedule(Collection stages)
{
requireNonNull(stages, "stages is null");
List preferredScheduleOrder = getPreferredScheduleOrder(stages.stream()
+ .map(StageExecutionAndScheduler::getStageExecution)
.map(SqlStageExecution::getFragment)
.collect(toImmutableList()));
- Ordering ordering = Ordering.explicit(preferredScheduleOrder)
+ Ordering ordering = Ordering.explicit(preferredScheduleOrder)
.onResultOf(PlanFragment::getId)
- .onResultOf(SqlStageExecution::getFragment);
+ .onResultOf(execution -> execution.getStageExecution().getFragment());
schedulingStages = new LinkedHashSet<>(ordering.sortedCopy(stages));
}
@Override
- public Set getStagesToSchedule()
+ public Set getStagesToSchedule()
{
- for (Iterator iterator = schedulingStages.iterator(); iterator.hasNext(); ) {
- StageExecutionState state = iterator.next().getState();
+ for (Iterator iterator = schedulingStages.iterator(); iterator.hasNext(); ) {
+ StageExecutionState state = iterator.next().getStageExecution().getState();
if (state == SCHEDULED || state == RUNNING || state.isDone()) {
iterator.remove();
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExchangeLocationsConsumer.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExchangeLocationsConsumer.java
new file mode 100644
index 0000000000000..695d7ad19f506
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExchangeLocationsConsumer.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.execution.RemoteTask;
+import com.facebook.presto.sql.planner.plan.PlanFragmentId;
+
+import java.util.Set;
+
+public interface ExchangeLocationsConsumer
+{
+ void addExchangeLocations(PlanFragmentId fragmentId, Set tasks, boolean noMoreExchangeLocations);
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionPolicy.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionPolicy.java
index 689685ed9600c..a6cc6a4ad7783 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionPolicy.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionPolicy.java
@@ -13,11 +13,9 @@
*/
package com.facebook.presto.execution.scheduler;
-import com.facebook.presto.execution.SqlStageExecution;
-
import java.util.Collection;
public interface ExecutionPolicy
{
- ExecutionSchedule createExecutionSchedule(Collection stages);
+ ExecutionSchedule createExecutionSchedule(Collection stages);
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionSchedule.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionSchedule.java
index fbd9b2e0a0d3f..81571db6c26fb 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionSchedule.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/ExecutionSchedule.java
@@ -13,13 +13,11 @@
*/
package com.facebook.presto.execution.scheduler;
-import com.facebook.presto.execution.SqlStageExecution;
-
import java.util.Set;
public interface ExecutionSchedule
{
- Set getStagesToSchedule();
+ Set getStagesToSchedule();
boolean isFinished();
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/FixedSourcePartitionedScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/FixedSourcePartitionedScheduler.java
index 353c692a126a6..b264a64f8d798 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/FixedSourcePartitionedScheduler.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/FixedSourcePartitionedScheduler.java
@@ -33,6 +33,8 @@
import com.google.common.collect.Streams;
import com.google.common.util.concurrent.ListenableFuture;
+import javax.annotation.concurrent.GuardedBy;
+
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -42,6 +44,7 @@
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.Supplier;
import static com.facebook.airlift.concurrent.MoreFutures.whenAnyComplete;
@@ -61,6 +64,7 @@ public class FixedSourcePartitionedScheduler
private final SqlStageExecution stage;
private final List nodes;
+
private final List sourceSchedulers;
private final List partitionHandles;
private boolean scheduledTasks;
@@ -69,6 +73,9 @@ public class FixedSourcePartitionedScheduler
private final Queue tasksToRecover = new ConcurrentLinkedQueue<>();
+ @GuardedBy("this")
+ private boolean closed;
+
public FixedSourcePartitionedScheduler(
SqlStageExecution stage,
Map splitSources,
@@ -154,7 +161,10 @@ public FixedSourcePartitionedScheduler(
}
}
this.groupedLifespanScheduler = groupedLifespanScheduler;
- this.sourceSchedulers = sourceSchedulers;
+
+ // use a CopyOnWriteArrayList to prevent ConcurrentModificationExceptions
+ // if close() is called while the main thread is in the scheduling loop
+ this.sourceSchedulers = new CopyOnWriteArrayList<>(sourceSchedulers);
}
private ConnectorPartitionHandle partitionHandleFor(Lifespan lifespan)
@@ -214,33 +224,40 @@ public ScheduleResult schedule()
Iterator schedulerIterator = sourceSchedulers.iterator();
List driverGroupsToStart = ImmutableList.of();
while (schedulerIterator.hasNext()) {
- SourceScheduler sourceScheduler = schedulerIterator.next();
+ synchronized (this) {
+ // if a source scheduler is closed while it is scheduling, we can get an error
+ // prevent that by checking if scheduling has been cancelled first.
+ if (closed) {
+ break;
+ }
+ SourceScheduler sourceScheduler = schedulerIterator.next();
- for (Lifespan lifespan : driverGroupsToStart) {
- sourceScheduler.startLifespan(lifespan, partitionHandleFor(lifespan));
- }
+ for (Lifespan lifespan : driverGroupsToStart) {
+ sourceScheduler.startLifespan(lifespan, partitionHandleFor(lifespan));
+ }
- ScheduleResult schedule = sourceScheduler.schedule();
- if (schedule.getSplitsScheduled() > 0) {
- stage.transitionToSchedulingSplits();
- }
- splitsScheduled += schedule.getSplitsScheduled();
- if (schedule.getBlockedReason().isPresent()) {
- blocked.add(schedule.getBlocked());
- blockedReason = blockedReason.combineWith(schedule.getBlockedReason().get());
- }
- else {
- verify(schedule.getBlocked().isDone(), "blockedReason not provided when scheduler is blocked");
- allBlocked = false;
- }
+ ScheduleResult schedule = sourceScheduler.schedule();
+ if (schedule.getSplitsScheduled() > 0) {
+ stage.transitionToSchedulingSplits();
+ }
+ splitsScheduled += schedule.getSplitsScheduled();
+ if (schedule.getBlockedReason().isPresent()) {
+ blocked.add(schedule.getBlocked());
+ blockedReason = blockedReason.combineWith(schedule.getBlockedReason().get());
+ }
+ else {
+ verify(schedule.getBlocked().isDone(), "blockedReason not provided when scheduler is blocked");
+ allBlocked = false;
+ }
- driverGroupsToStart = sourceScheduler.drainCompletelyScheduledLifespans();
+ driverGroupsToStart = sourceScheduler.drainCompletelyScheduledLifespans();
- if (schedule.isFinished()) {
- stage.schedulingComplete(sourceScheduler.getPlanNodeId());
- schedulerIterator.remove();
- sourceScheduler.close();
- anySourceSchedulingFinished = true;
+ if (schedule.isFinished()) {
+ stage.schedulingComplete(sourceScheduler.getPlanNodeId());
+ sourceSchedulers.remove(sourceScheduler);
+ sourceScheduler.close();
+ anySourceSchedulingFinished = true;
+ }
}
}
@@ -258,8 +275,9 @@ public void recover(TaskId taskId)
}
@Override
- public void close()
+ public synchronized void close()
{
+ closed = true;
for (SourceScheduler sourceScheduler : sourceSchedulers) {
try {
sourceScheduler.close();
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java
new file mode 100644
index 0000000000000..ceacc02d7e67d
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java
@@ -0,0 +1,579 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.airlift.concurrent.SetThreadName;
+import com.facebook.airlift.stats.TimeStat;
+import com.facebook.presto.Session;
+import com.facebook.presto.execution.BasicStageExecutionStats;
+import com.facebook.presto.execution.LocationFactory;
+import com.facebook.presto.execution.QueryState;
+import com.facebook.presto.execution.QueryStateMachine;
+import com.facebook.presto.execution.RemoteTask;
+import com.facebook.presto.execution.RemoteTaskFactory;
+import com.facebook.presto.execution.SqlStageExecution;
+import com.facebook.presto.execution.StageExecutionInfo;
+import com.facebook.presto.execution.StageExecutionState;
+import com.facebook.presto.execution.StageId;
+import com.facebook.presto.execution.StageInfo;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.buffer.OutputBuffers;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.facebook.presto.spi.PrestoException;
+import com.facebook.presto.sql.planner.PlanFragment;
+import com.facebook.presto.sql.planner.SplitSourceFactory;
+import com.facebook.presto.sql.planner.SubPlan;
+import com.facebook.presto.sql.planner.plan.PlanFragmentId;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.units.Duration;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.facebook.airlift.concurrent.MoreFutures.tryGetFutureValue;
+import static com.facebook.airlift.concurrent.MoreFutures.whenAnyComplete;
+import static com.facebook.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
+import static com.facebook.presto.SystemSessionProperties.getMaxConcurrentMaterializations;
+import static com.facebook.presto.execution.BasicStageExecutionStats.aggregateBasicStageStats;
+import static com.facebook.presto.execution.StageExecutionState.ABORTED;
+import static com.facebook.presto.execution.StageExecutionState.CANCELED;
+import static com.facebook.presto.execution.StageExecutionState.FAILED;
+import static com.facebook.presto.execution.StageExecutionState.FINISHED;
+import static com.facebook.presto.execution.StageExecutionState.PLANNED;
+import static com.facebook.presto.execution.StageExecutionState.RUNNING;
+import static com.facebook.presto.execution.StageExecutionState.SCHEDULED;
+import static com.facebook.presto.execution.buffer.OutputBuffers.createDiscardingOutputBuffers;
+import static com.facebook.presto.execution.scheduler.StreamingPlanSection.extractStreamingSections;
+import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.collect.ImmutableMap.toImmutableMap;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static com.google.common.collect.Streams.stream;
+import static com.google.common.graph.Traverser.forTree;
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.function.Function.identity;
+
+@Deprecated
+public class LegacySqlQueryScheduler
+ implements SqlQuerySchedulerInterface
+{
+ private final LocationFactory locationFactory;
+ private final ExecutionPolicy executionPolicy;
+
+ private final SplitSchedulerStats schedulerStats;
+
+ private final QueryStateMachine queryStateMachine;
+ private final SubPlan plan;
+ private final StreamingPlanSection sectionedPlan;
+ private final StageId rootStageId;
+ private final boolean summarizeTaskInfo;
+ private final int maxConcurrentMaterializations;
+
+ private final Map stageExecutions;
+ private final ExecutorService executor;
+ private final AtomicBoolean started = new AtomicBoolean();
+ private final AtomicBoolean scheduling = new AtomicBoolean();
+
+ public static LegacySqlQueryScheduler createSqlQueryScheduler(
+ LocationFactory locationFactory,
+ ExecutionPolicy executionPolicy,
+ ExecutorService queryExecutor,
+ SplitSchedulerStats schedulerStats,
+ SectionExecutionFactory sectionExecutionFactory,
+ RemoteTaskFactory remoteTaskFactory,
+ SplitSourceFactory splitSourceFactory,
+ Session session,
+ QueryStateMachine queryStateMachine,
+ SubPlan plan,
+ OutputBuffers rootOutputBuffers,
+ boolean summarizeTaskInfo)
+ {
+ LegacySqlQueryScheduler sqlQueryScheduler = new LegacySqlQueryScheduler(
+ locationFactory,
+ executionPolicy,
+ queryExecutor,
+ schedulerStats,
+ sectionExecutionFactory,
+ remoteTaskFactory,
+ splitSourceFactory,
+ session,
+ queryStateMachine,
+ plan,
+ summarizeTaskInfo,
+ rootOutputBuffers);
+ sqlQueryScheduler.initialize();
+ return sqlQueryScheduler;
+ }
+
+ private LegacySqlQueryScheduler(
+ LocationFactory locationFactory,
+ ExecutionPolicy executionPolicy,
+ ExecutorService queryExecutor,
+ SplitSchedulerStats schedulerStats,
+ SectionExecutionFactory sectionExecutionFactory,
+ RemoteTaskFactory remoteTaskFactory,
+ SplitSourceFactory splitSourceFactory,
+ Session session,
+ QueryStateMachine queryStateMachine,
+ SubPlan plan,
+ boolean summarizeTaskInfo,
+ OutputBuffers rootOutputBuffers)
+ {
+ this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
+ this.executionPolicy = requireNonNull(executionPolicy, "schedulerPolicyFactory is null");
+ this.executor = queryExecutor;
+ this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null");
+ this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null");
+ this.plan = requireNonNull(plan, "plan is null");
+ this.sectionedPlan = extractStreamingSections(plan);
+ this.summarizeTaskInfo = summarizeTaskInfo;
+
+ OutputBufferId rootBufferId = getOnlyElement(rootOutputBuffers.getBuffers().keySet());
+ List stageExecutions = createStageExecutions(
+ sectionExecutionFactory,
+ (fragmentId, tasks, noMoreExchangeLocations) -> updateQueryOutputLocations(queryStateMachine, rootBufferId, tasks, noMoreExchangeLocations),
+ sectionedPlan,
+ Optional.of(new int[1]),
+ rootOutputBuffers,
+ remoteTaskFactory,
+ splitSourceFactory,
+ session);
+
+ this.rootStageId = Iterables.getLast(stageExecutions).getStageExecution().getStageExecutionId().getStageId();
+
+ this.stageExecutions = stageExecutions.stream()
+ .collect(toImmutableMap(execution -> execution.getStageExecution().getStageExecutionId().getStageId(), identity()));
+
+ this.maxConcurrentMaterializations = getMaxConcurrentMaterializations(session);
+ }
+
+ // this is a separate method to ensure that the `this` reference is not leaked during construction
+ private void initialize()
+ {
+ SqlStageExecution rootStage = stageExecutions.get(rootStageId).getStageExecution();
+ rootStage.addStateChangeListener(state -> {
+ if (state == FINISHED) {
+ queryStateMachine.transitionToFinishing();
+ }
+ else if (state == CANCELED) {
+ // output stage was canceled
+ queryStateMachine.transitionToCanceled();
+ }
+ });
+
+ for (StageExecutionAndScheduler stageExecutionInfo : stageExecutions.values()) {
+ SqlStageExecution stageExecution = stageExecutionInfo.getStageExecution();
+ stageExecution.addStateChangeListener(state -> {
+ if (queryStateMachine.isDone()) {
+ return;
+ }
+ if (state == FAILED) {
+ queryStateMachine.transitionToFailed(stageExecution.getStageExecutionInfo().getFailureCause().get().toException());
+ }
+ else if (state == ABORTED) {
+ // this should never happen, since abort can only be triggered in query clean up after the query is finished
+ queryStateMachine.transitionToFailed(new PrestoException(GENERIC_INTERNAL_ERROR, "Query stage was aborted"));
+ }
+ else if (state == FINISHED) {
+ // checks if there's any new sections available for execution and starts the scheduling if any
+ startScheduling();
+ }
+ else if (queryStateMachine.getQueryState() == QueryState.STARTING) {
+ // if the stage has at least one task, we are running
+ if (stageExecution.hasTasks()) {
+ queryStateMachine.transitionToRunning();
+ }
+ }
+ });
+ stageExecution.addFinalStageInfoListener(status -> queryStateMachine.updateQueryInfo(Optional.of(getStageInfo())));
+ }
+
+ // when query is done or any time a stage completes, attempt to transition query to "final query info ready"
+ queryStateMachine.addStateChangeListener(newState -> {
+ if (newState.isDone()) {
+ queryStateMachine.updateQueryInfo(Optional.of(getStageInfo()));
+ }
+ });
+ }
+
+ private static void updateQueryOutputLocations(QueryStateMachine queryStateMachine, OutputBufferId rootBufferId, Set tasks, boolean noMoreExchangeLocations)
+ {
+ Map bufferLocations = tasks.stream()
+ .collect(toImmutableMap(
+ task -> getBufferLocation(task, rootBufferId),
+ RemoteTask::getTaskId));
+ queryStateMachine.updateOutputLocations(bufferLocations, noMoreExchangeLocations);
+ }
+
+ private static URI getBufferLocation(RemoteTask remoteTask, OutputBufferId rootBufferId)
+ {
+ URI location = remoteTask.getTaskStatus().getSelf();
+ return uriBuilderFrom(location).appendPath("results").appendPath(rootBufferId.toString()).build();
+ }
+
+ /**
+ * returns a List of SqlStageExecutionInfos in a postorder representation of the tree
+ */
+ private List createStageExecutions(
+ SectionExecutionFactory sectionExecutionFactory,
+ ExchangeLocationsConsumer locationsConsumer,
+ StreamingPlanSection section,
+ Optional bucketToPartition,
+ OutputBuffers outputBuffers,
+ RemoteTaskFactory remoteTaskFactory,
+ SplitSourceFactory splitSourceFactory,
+ Session session)
+ {
+ ImmutableList.Builder stages = ImmutableList.builder();
+
+ for (StreamingPlanSection childSection : section.getChildren()) {
+ ExchangeLocationsConsumer childLocationsConsumer = (fragmentId, tasks, noMoreExhchangeLocations) -> {};
+ stages.addAll(createStageExecutions(
+ sectionExecutionFactory,
+ childLocationsConsumer,
+ childSection,
+ Optional.empty(),
+ createDiscardingOutputBuffers(),
+ remoteTaskFactory,
+ splitSourceFactory,
+ session));
+ }
+ List sectionStages =
+ sectionExecutionFactory.createSectionExecutions(
+ session,
+ section,
+ locationsConsumer,
+ bucketToPartition,
+ outputBuffers,
+ summarizeTaskInfo,
+ remoteTaskFactory,
+ splitSourceFactory,
+ 0).getSectionStages();
+ stages.addAll(sectionStages);
+
+ return stages.build();
+ }
+
+ public void start()
+ {
+ if (started.compareAndSet(false, true)) {
+ startScheduling();
+ }
+ }
+
+ private void startScheduling()
+ {
+ requireNonNull(stageExecutions);
+ // still scheduling the previous batch of stages
+ if (scheduling.get()) {
+ return;
+ }
+ executor.submit(this::schedule);
+ }
+
+ private void schedule()
+ {
+ if (!scheduling.compareAndSet(false, true)) {
+ // still scheduling the previous batch of stages
+ return;
+ }
+
+ List scheduledStageExecutions = new ArrayList<>();
+
+ try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
+ Set completedStages = new HashSet<>();
+
+ List sectionExecutionSchedules = new LinkedList<>();
+
+ while (!Thread.currentThread().isInterrupted()) {
+ // remove finished section
+ sectionExecutionSchedules.removeIf(ExecutionSchedule::isFinished);
+
+ // try to pull more section that are ready to be run
+ List sectionsReadyForExecution = getSectionsReadyForExecution();
+
+ // all finished
+ if (sectionsReadyForExecution.isEmpty() && sectionExecutionSchedules.isEmpty()) {
+ break;
+ }
+
+ List> sectionStageExecutions = getStageExecutions(sectionsReadyForExecution);
+ sectionStageExecutions.forEach(scheduledStageExecutions::addAll);
+ sectionStageExecutions.stream()
+ .map(executionInfos -> executionInfos.stream()
+ .collect(toImmutableList()))
+ .map(executionPolicy::createExecutionSchedule)
+ .forEach(sectionExecutionSchedules::add);
+
+ while (sectionExecutionSchedules.stream().noneMatch(ExecutionSchedule::isFinished)) {
+ List> blockedStages = new ArrayList<>();
+
+ List executionsToSchedule = sectionExecutionSchedules.stream()
+ .flatMap(schedule -> schedule.getStagesToSchedule().stream())
+ .collect(toImmutableList());
+
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : executionsToSchedule) {
+ SqlStageExecution stageExecution = stageExecutionAndScheduler.getStageExecution();
+ StageId stageId = stageExecution.getStageExecutionId().getStageId();
+ stageExecution.beginScheduling();
+
+ // perform some scheduling work
+ ScheduleResult result = stageExecutionAndScheduler.getStageScheduler()
+ .schedule();
+
+ // modify parent and children based on the results of the scheduling
+ if (result.isFinished()) {
+ stageExecution.schedulingComplete();
+ }
+ else if (!result.getBlocked().isDone()) {
+ blockedStages.add(result.getBlocked());
+ }
+ stageExecutionAndScheduler.getStageLinkage()
+ .processScheduleResults(stageExecution.getState(), result.getNewTasks());
+ schedulerStats.getSplitsScheduledPerIteration().add(result.getSplitsScheduled());
+ if (result.getBlockedReason().isPresent()) {
+ switch (result.getBlockedReason().get()) {
+ case WRITER_SCALING:
+ // no-op
+ break;
+ case WAITING_FOR_SOURCE:
+ schedulerStats.getWaitingForSource().update(1);
+ break;
+ case SPLIT_QUEUES_FULL:
+ schedulerStats.getSplitQueuesFull().update(1);
+ break;
+ case MIXED_SPLIT_QUEUES_FULL_AND_WAITING_FOR_SOURCE:
+ schedulerStats.getMixedSplitQueuesFullAndWaitingForSource().update(1);
+ break;
+ case NO_ACTIVE_DRIVER_GROUP:
+ schedulerStats.getNoActiveDriverGroup().update(1);
+ break;
+ default:
+ throw new UnsupportedOperationException("Unknown blocked reason: " + result.getBlockedReason().get());
+ }
+ }
+ }
+
+ // make sure to update stage linkage at least once per loop to catch async state changes (e.g., partial cancel)
+ boolean stageFinishedExecution = false;
+ for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
+ SqlStageExecution stageExecution = stageExecutionInfo.getStageExecution();
+ StageId stageId = stageExecution.getStageExecutionId().getStageId();
+ if (!completedStages.contains(stageId) && stageExecution.getState().isDone()) {
+ stageExecutionInfo.getStageLinkage()
+ .processScheduleResults(stageExecution.getState(), ImmutableSet.of());
+ completedStages.add(stageId);
+ stageFinishedExecution = true;
+ }
+ }
+
+ // if any stage has just finished execution try to pull more sections for scheduling
+ if (stageFinishedExecution) {
+ break;
+ }
+
+ // wait for a state change and then schedule again
+ if (!blockedStages.isEmpty()) {
+ try (TimeStat.BlockTimer timer = schedulerStats.getSleepTime().time()) {
+ tryGetFutureValue(whenAnyComplete(blockedStages), 1, SECONDS);
+ }
+ for (ListenableFuture> blockedStage : blockedStages) {
+ blockedStage.cancel(true);
+ }
+ }
+ }
+ }
+
+ for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
+ StageExecutionState state = stageExecutionInfo.getStageExecution().getState();
+ if (state != SCHEDULED && state != RUNNING && !state.isDone()) {
+ throw new PrestoException(GENERIC_INTERNAL_ERROR, format("Scheduling is complete, but stage execution %s is in state %s", stageExecutionInfo.getStageExecution().getStageExecutionId(), state));
+ }
+ }
+
+ scheduling.set(false);
+
+ if (!getSectionsReadyForExecution().isEmpty()) {
+ startScheduling();
+ }
+ }
+ catch (Throwable t) {
+ scheduling.set(false);
+ queryStateMachine.transitionToFailed(t);
+ throw t;
+ }
+ finally {
+ RuntimeException closeError = new RuntimeException();
+ for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
+ try {
+ stageExecutionInfo.getStageScheduler().close();
+ }
+ catch (Throwable t) {
+ queryStateMachine.transitionToFailed(t);
+ // Self-suppression not permitted
+ if (closeError != t) {
+ closeError.addSuppressed(t);
+ }
+ }
+ }
+ if (closeError.getSuppressed().length > 0) {
+ throw closeError;
+ }
+ }
+ }
+
+ private List getSectionsReadyForExecution()
+ {
+ long runningPlanSections =
+ stream(forTree(StreamingPlanSection::getChildren).depthFirstPreOrder(sectionedPlan))
+ .map(section -> getStageExecution(section.getPlan().getFragment().getId()).getState())
+ .filter(state -> !state.isDone() && state != PLANNED)
+ .count();
+ return stream(forTree(StreamingPlanSection::getChildren).depthFirstPreOrder(sectionedPlan))
+ // get all sections ready for execution
+ .filter(this::isReadyForExecution)
+ .limit(maxConcurrentMaterializations - runningPlanSections)
+ .collect(toImmutableList());
+ }
+
+ private boolean isReadyForExecution(StreamingPlanSection section)
+ {
+ SqlStageExecution stageExecution = getStageExecution(section.getPlan().getFragment().getId());
+ if (stageExecution.getState() != PLANNED) {
+ // already scheduled
+ return false;
+ }
+ for (StreamingPlanSection child : section.getChildren()) {
+ SqlStageExecution rootStageExecution = getStageExecution(child.getPlan().getFragment().getId());
+ if (rootStageExecution.getState() != FINISHED) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private List> getStageExecutions(List sections)
+ {
+ return sections.stream()
+ .map(section -> stream(forTree(StreamingSubPlan::getChildren).depthFirstPreOrder(section.getPlan())).collect(toImmutableList()))
+ .map(plans -> plans.stream()
+ .map(StreamingSubPlan::getFragment)
+ .map(PlanFragment::getId)
+ .map(this::getStageExecutionInfo)
+ .collect(toImmutableList()))
+ .collect(toImmutableList());
+ }
+
+ private SqlStageExecution getStageExecution(PlanFragmentId planFragmentId)
+ {
+ return stageExecutions.get(getStageId(planFragmentId)).getStageExecution();
+ }
+
+ private StageExecutionAndScheduler getStageExecutionInfo(PlanFragmentId planFragmentId)
+ {
+ return stageExecutions.get(getStageId(planFragmentId));
+ }
+
+ private StageId getStageId(PlanFragmentId fragmentId)
+ {
+ return new StageId(queryStateMachine.getQueryId(), fragmentId.getId());
+ }
+
+ public long getUserMemoryReservation()
+ {
+ return stageExecutions.values().stream()
+ .mapToLong(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getUserMemoryReservation())
+ .sum();
+ }
+
+ public long getTotalMemoryReservation()
+ {
+ return stageExecutions.values().stream()
+ .mapToLong(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getTotalMemoryReservation())
+ .sum();
+ }
+
+ public Duration getTotalCpuTime()
+ {
+ long millis = stageExecutions.values().stream()
+ .mapToLong(stage -> stage.getStageExecution().getTotalCpuTime().toMillis())
+ .sum();
+ return new Duration(millis, MILLISECONDS);
+ }
+
+ public BasicStageExecutionStats getBasicStageStats()
+ {
+ List stageStats = stageExecutions.values().stream()
+ .map(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getBasicStageStats())
+ .collect(toImmutableList());
+
+ return aggregateBasicStageStats(stageStats);
+ }
+
+ public StageInfo getStageInfo()
+ {
+ Map stageInfos = stageExecutions.values().stream()
+ .map(StageExecutionAndScheduler::getStageExecution)
+ .collect(toImmutableMap(execution -> execution.getStageExecutionId().getStageId(), SqlStageExecution::getStageExecutionInfo));
+
+ return buildStageInfo(plan, stageInfos);
+ }
+
+ private StageInfo buildStageInfo(SubPlan subPlan, Map stageExecutionInfos)
+ {
+ StageId stageId = getStageId(subPlan.getFragment().getId());
+ StageExecutionInfo stageExecutionInfo = stageExecutionInfos.get(stageId);
+ checkArgument(stageExecutionInfo != null, "No stageExecutionInfo for %s", stageId);
+ return new StageInfo(
+ stageId,
+ locationFactory.createStageLocation(stageId),
+ Optional.of(subPlan.getFragment()),
+ stageExecutionInfo,
+ ImmutableList.of(),
+ subPlan.getChildren().stream()
+ .map(plan -> buildStageInfo(plan, stageExecutionInfos))
+ .collect(toImmutableList()));
+ }
+
+ public void cancelStage(StageId stageId)
+ {
+ try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
+ SqlStageExecution execution = stageExecutions.get(stageId).getStageExecution();
+ SqlStageExecution stage = requireNonNull(execution, () -> format("Stage %s does not exist", stageId));
+ stage.cancel();
+ }
+ }
+
+ public void abort()
+ {
+ try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
+ stageExecutions.values().forEach(stageExecutionInfo -> stageExecutionInfo.getStageExecution().abort());
+ }
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionPolicy.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionPolicy.java
index b2fcf2a0caf3b..40d2c9b329f73 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionPolicy.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionPolicy.java
@@ -13,15 +13,13 @@
*/
package com.facebook.presto.execution.scheduler;
-import com.facebook.presto.execution.SqlStageExecution;
-
import java.util.Collection;
public class PhasedExecutionPolicy
implements ExecutionPolicy
{
@Override
- public ExecutionSchedule createExecutionSchedule(Collection stages)
+ public ExecutionSchedule createExecutionSchedule(Collection stages)
{
return new PhasedExecutionSchedule(stages);
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionSchedule.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionSchedule.java
index 2f31f9f912d89..d9a234e18e9b4 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionSchedule.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/PhasedExecutionSchedule.java
@@ -58,14 +58,17 @@
public class PhasedExecutionSchedule
implements ExecutionSchedule
{
- private final List> schedulePhases;
- private final Set activeSources = new HashSet<>();
+ private final List> schedulePhases;
+ private final Set activeSources = new HashSet<>();
- public PhasedExecutionSchedule(Collection stages)
+ public PhasedExecutionSchedule(Collection stages)
{
- List> phases = extractPhases(stages.stream().map(SqlStageExecution::getFragment).collect(toImmutableList()));
+ List> phases = extractPhases(stages.stream()
+ .map(StageExecutionAndScheduler::getStageExecution)
+ .map(SqlStageExecution::getFragment)
+ .collect(toImmutableList()));
- Map stagesByFragmentId = stages.stream().collect(toImmutableMap(stage -> stage.getFragment().getId(), identity()));
+ Map stagesByFragmentId = stages.stream().collect(toImmutableMap(stage -> stage.getStageExecution().getFragment().getId(), identity()));
// create a mutable list of mutable sets of stages, so we can remove completed stages
schedulePhases = new ArrayList<>();
@@ -77,7 +80,7 @@ public PhasedExecutionSchedule(Collection stages)
}
@Override
- public Set getStagesToSchedule()
+ public Set getStagesToSchedule()
{
removeCompletedStages();
addPhasesIfNecessary();
@@ -89,8 +92,8 @@ public Set getStagesToSchedule()
private void removeCompletedStages()
{
- for (Iterator stageIterator = activeSources.iterator(); stageIterator.hasNext(); ) {
- StageExecutionState state = stageIterator.next().getState();
+ for (Iterator stageIterator = activeSources.iterator(); stageIterator.hasNext(); ) {
+ StageExecutionState state = stageIterator.next().getStageExecution().getState();
if (state == SCHEDULED || state == RUNNING || state.isDone()) {
stageIterator.remove();
}
@@ -105,7 +108,7 @@ private void addPhasesIfNecessary()
}
while (!schedulePhases.isEmpty()) {
- Set phase = schedulePhases.remove(0);
+ Set phase = schedulePhases.remove(0);
activeSources.addAll(phase);
if (hasSourceDistributedStage(phase)) {
return;
@@ -113,9 +116,9 @@ private void addPhasesIfNecessary()
}
}
- private static boolean hasSourceDistributedStage(Set phase)
+ private static boolean hasSourceDistributedStage(Set phase)
{
- return phase.stream().anyMatch(stage -> !stage.getFragment().getTableScanSchedulingOrder().isEmpty());
+ return phase.stream().anyMatch(stage -> !stage.getStageExecution().getFragment().getTableScanSchedulingOrder().isEmpty());
}
@Override
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecution.java
new file mode 100644
index 0000000000000..a829aa8542070
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecution.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.execution.StageExecutionState;
+import com.google.common.collect.ImmutableList;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+
+import static com.facebook.presto.execution.StageExecutionState.PLANNED;
+import static java.util.Objects.requireNonNull;
+
+public class SectionExecution
+{
+ private final StageExecutionAndScheduler rootStage;
+ private final List allStages;
+
+ @GuardedBy("this")
+ private volatile boolean aborted;
+
+ public SectionExecution(StageExecutionAndScheduler rootStage, List allStages)
+ {
+ this.rootStage = requireNonNull(rootStage, "rootStage is null");
+ this.allStages = ImmutableList.copyOf(requireNonNull(allStages, "allStages is null"));
+ }
+
+ public StageExecutionAndScheduler getRootStage()
+ {
+ return rootStage;
+ }
+
+ public List getSectionStages()
+ {
+ return allStages;
+ }
+
+ public boolean isFinished()
+ {
+ StageExecutionState rootStageState = rootStage.getStageExecution().getState();
+ return rootStageState.isDone() && !rootStageState.isFailure();
+ }
+
+ public boolean isFailed()
+ {
+ StageExecutionState rootStageState = rootStage.getStageExecution().getState();
+ return rootStageState.isFailure();
+ }
+
+ public boolean isRunning()
+ {
+ StageExecutionState rootStageState = rootStage.getStageExecution().getState();
+ return !rootStageState.isDone() && rootStageState != PLANNED;
+ }
+
+ public synchronized boolean abort()
+ {
+ if (!aborted) {
+ aborted = true;
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : allStages) {
+ stageExecutionAndScheduler.getStageExecution().abort();
+ stageExecutionAndScheduler.getStageScheduler().close();
+ }
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecutionFactory.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecutionFactory.java
new file mode 100644
index 0000000000000..04e0e73002017
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SectionExecutionFactory.java
@@ -0,0 +1,440 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.Session;
+import com.facebook.presto.execution.ForQueryExecution;
+import com.facebook.presto.execution.NodeTaskMap;
+import com.facebook.presto.execution.QueryManagerConfig;
+import com.facebook.presto.execution.RemoteTask;
+import com.facebook.presto.execution.RemoteTaskFactory;
+import com.facebook.presto.execution.SqlStageExecution;
+import com.facebook.presto.execution.StageExecutionId;
+import com.facebook.presto.execution.StageExecutionState;
+import com.facebook.presto.execution.StageId;
+import com.facebook.presto.execution.TaskStatus;
+import com.facebook.presto.execution.buffer.OutputBuffers;
+import com.facebook.presto.failureDetector.FailureDetector;
+import com.facebook.presto.metadata.InternalNode;
+import com.facebook.presto.metadata.Metadata;
+import com.facebook.presto.operator.ForScheduler;
+import com.facebook.presto.spi.ConnectorId;
+import com.facebook.presto.spi.connector.ConnectorPartitionHandle;
+import com.facebook.presto.spi.plan.PlanNode;
+import com.facebook.presto.spi.plan.PlanNodeId;
+import com.facebook.presto.spi.plan.TableScanNode;
+import com.facebook.presto.split.SplitSource;
+import com.facebook.presto.sql.planner.NodePartitionMap;
+import com.facebook.presto.sql.planner.NodePartitioningManager;
+import com.facebook.presto.sql.planner.PartitioningHandle;
+import com.facebook.presto.sql.planner.SplitSourceFactory;
+import com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher;
+import com.facebook.presto.sql.planner.plan.PlanFragmentId;
+import com.facebook.presto.sql.planner.plan.RemoteSourceNode;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+import javax.inject.Inject;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import static com.facebook.presto.SystemSessionProperties.getConcurrentLifespansPerNode;
+import static com.facebook.presto.SystemSessionProperties.getMaxTasksPerStage;
+import static com.facebook.presto.SystemSessionProperties.getWriterMinSize;
+import static com.facebook.presto.execution.SqlStageExecution.createSqlStageExecution;
+import static com.facebook.presto.execution.scheduler.SourcePartitionedScheduler.newSourcePartitionedSchedulerAsStageScheduler;
+import static com.facebook.presto.execution.scheduler.TableWriteInfo.createTableWriteInfo;
+import static com.facebook.presto.spi.ConnectorId.isInternalSystemConnector;
+import static com.facebook.presto.spi.StandardErrorCode.NO_NODES_AVAILABLE;
+import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
+import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION;
+import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION;
+import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.REPLICATE;
+import static com.facebook.presto.util.Failures.checkCondition;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.collect.Iterables.getLast;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static com.google.common.collect.Sets.newConcurrentHashSet;
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+
+public class SectionExecutionFactory
+{
+ private final Metadata metadata;
+ private final NodePartitioningManager nodePartitioningManager;
+ private final NodeTaskMap nodeTaskMap;
+ private final ExecutorService executor;
+ private final ScheduledExecutorService scheduledExecutor;
+ private final FailureDetector failureDetector;
+ private final SplitSchedulerStats schedulerStats;
+ private final NodeScheduler nodeScheduler;
+ private final int splitBatchSize;
+
+ @Inject
+ public SectionExecutionFactory(
+ Metadata metadata,
+ NodePartitioningManager nodePartitioningManager,
+ NodeTaskMap nodeTaskMap,
+ @ForQueryExecution ExecutorService executor,
+ @ForScheduler ScheduledExecutorService scheduledExecutor,
+ FailureDetector failureDetector,
+ SplitSchedulerStats schedulerStats,
+ NodeScheduler nodeScheduler,
+ QueryManagerConfig queryManagerConfig)
+ {
+ this(
+ metadata,
+ nodePartitioningManager,
+ nodeTaskMap,
+ executor,
+ scheduledExecutor,
+ failureDetector,
+ schedulerStats,
+ nodeScheduler,
+ requireNonNull(queryManagerConfig, "queryManagerConfig is null").getScheduleSplitBatchSize());
+ }
+
+ public SectionExecutionFactory(
+ Metadata metadata,
+ NodePartitioningManager nodePartitioningManager,
+ NodeTaskMap nodeTaskMap,
+ ExecutorService executor,
+ ScheduledExecutorService scheduledExecutor,
+ FailureDetector failureDetector,
+ SplitSchedulerStats schedulerStats,
+ NodeScheduler nodeScheduler,
+ int splitBatchSize)
+ {
+ this.metadata = requireNonNull(metadata, "metadata is null");
+ this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null");
+ this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null");
+ this.executor = requireNonNull(executor, "executor is null");
+ this.scheduledExecutor = requireNonNull(scheduledExecutor, "scheduledExecutor is null");
+ this.failureDetector = requireNonNull(failureDetector, "failureDetector is null");
+ this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null");
+ this.nodeScheduler = requireNonNull(nodeScheduler, "nodeScheduler is null");
+ this.splitBatchSize = splitBatchSize;
+ }
+
+ /**
+ * returns a List of SectionExecutions in a postorder representation of the tree
+ */
+ public SectionExecution createSectionExecutions(
+ Session session,
+ StreamingPlanSection section,
+ ExchangeLocationsConsumer locationsConsumer,
+ Optional bucketToPartition,
+ OutputBuffers outputBuffers,
+ boolean summarizeTaskInfo,
+ RemoteTaskFactory remoteTaskFactory,
+ SplitSourceFactory splitSourceFactory,
+ int attemptId)
+ {
+ // Only fetch a distribution once per section to ensure all stages see the same machine assignments
+ Map partitioningCache = new HashMap<>();
+ TableWriteInfo tableWriteInfo = createTableWriteInfo(section.getPlan(), metadata, session);
+ List sectionStages = createStreamingLinkedStageExecutions(
+ session,
+ locationsConsumer,
+ section.getPlan().withBucketToPartition(bucketToPartition),
+ partitioningHandle -> partitioningCache.computeIfAbsent(partitioningHandle, handle -> nodePartitioningManager.getNodePartitioningMap(session, handle)),
+ tableWriteInfo,
+ Optional.empty(),
+ summarizeTaskInfo,
+ remoteTaskFactory,
+ splitSourceFactory,
+ attemptId);
+ StageExecutionAndScheduler rootStage = getLast(sectionStages);
+ rootStage.getStageExecution().setOutputBuffers(outputBuffers);
+ return new SectionExecution(rootStage, sectionStages);
+ }
+
+ /**
+ * returns a List of StageExecutionAndSchedulers in a postorder representation of the tree
+ */
+ private List createStreamingLinkedStageExecutions(
+ Session session,
+ ExchangeLocationsConsumer parent,
+ StreamingSubPlan plan,
+ Function partitioningCache,
+ TableWriteInfo tableWriteInfo,
+ Optional parentStageExecution,
+ boolean summarizeTaskInfo,
+ RemoteTaskFactory remoteTaskFactory,
+ SplitSourceFactory splitSourceFactory,
+ int attemptId)
+ {
+ ImmutableList.Builder stageExecutionAndSchedulers = ImmutableList.builder();
+
+ PlanFragmentId fragmentId = plan.getFragment().getId();
+ StageId stageId = new StageId(session.getQueryId(), fragmentId.getId());
+ SqlStageExecution stageExecution = createSqlStageExecution(
+ new StageExecutionId(stageId, attemptId),
+ plan.getFragment(),
+ remoteTaskFactory,
+ session,
+ summarizeTaskInfo,
+ nodeTaskMap,
+ executor,
+ failureDetector,
+ schedulerStats,
+ tableWriteInfo);
+
+ PartitioningHandle partitioningHandle = plan.getFragment().getPartitioning();
+ List remoteSourceNodes = plan.getFragment().getRemoteSourceNodes();
+ Optional bucketToPartition = getBucketToPartition(partitioningHandle, partitioningCache, plan.getFragment().getRoot(), remoteSourceNodes);
+
+ // create child stages
+ ImmutableSet.Builder childStagesBuilder = ImmutableSet.builder();
+ for (StreamingSubPlan stagePlan : plan.getChildren()) {
+ List subTree = createStreamingLinkedStageExecutions(
+ session,
+ stageExecution::addExchangeLocations,
+ stagePlan.withBucketToPartition(bucketToPartition),
+ partitioningCache,
+ tableWriteInfo,
+ Optional.of(stageExecution),
+ summarizeTaskInfo,
+ remoteTaskFactory,
+ splitSourceFactory,
+ attemptId);
+ stageExecutionAndSchedulers.addAll(subTree);
+ childStagesBuilder.add(getLast(subTree).getStageExecution());
+ }
+ Set childStageExecutions = childStagesBuilder.build();
+ stageExecution.addStateChangeListener(newState -> {
+ if (newState.isDone()) {
+ childStageExecutions.forEach(SqlStageExecution::cancel);
+ }
+ });
+
+ StageLinkage stageLinkage = new StageLinkage(fragmentId, parent, childStageExecutions);
+ StageScheduler stageScheduler = createStageScheduler(
+ splitSourceFactory,
+ session,
+ plan,
+ partitioningCache,
+ parentStageExecution,
+ stageId,
+ stageExecution,
+ partitioningHandle,
+ tableWriteInfo,
+ childStageExecutions);
+ stageExecutionAndSchedulers.add(new StageExecutionAndScheduler(
+ stageExecution,
+ stageLinkage,
+ stageScheduler));
+
+ return stageExecutionAndSchedulers.build();
+ }
+
+ private StageScheduler createStageScheduler(
+ SplitSourceFactory splitSourceFactory,
+ Session session,
+ StreamingSubPlan plan,
+ Function partitioningCache,
+ Optional parentStageExecution,
+ StageId stageId,
+ SqlStageExecution stageExecution,
+ PartitioningHandle partitioningHandle,
+ TableWriteInfo tableWriteInfo,
+ Set childStageExecutions)
+ {
+ Map splitSources = splitSourceFactory.createSplitSources(plan.getFragment(), session, tableWriteInfo);
+ int maxTasksPerStage = getMaxTasksPerStage(session);
+ if (partitioningHandle.equals(SOURCE_DISTRIBUTION)) {
+ // nodes are selected dynamically based on the constraints of the splits and the system load
+ Map.Entry entry = getOnlyElement(splitSources.entrySet());
+ PlanNodeId planNodeId = entry.getKey();
+ SplitSource splitSource = entry.getValue();
+ ConnectorId connectorId = splitSource.getConnectorId();
+ if (isInternalSystemConnector(connectorId)) {
+ connectorId = null;
+ }
+
+ NodeSelector nodeSelector = nodeScheduler.createNodeSelector(connectorId, maxTasksPerStage);
+ SplitPlacementPolicy placementPolicy = new DynamicSplitPlacementPolicy(nodeSelector, stageExecution::getAllTasks);
+
+ checkArgument(!plan.getFragment().getStageExecutionDescriptor().isStageGroupedExecution());
+ return newSourcePartitionedSchedulerAsStageScheduler(stageExecution, planNodeId, splitSource, placementPolicy, splitBatchSize);
+ }
+ else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
+ Supplier> sourceTasksProvider = () -> childStageExecutions.stream()
+ .map(SqlStageExecution::getAllTasks)
+ .flatMap(Collection::stream)
+ .map(RemoteTask::getTaskStatus)
+ .collect(toList());
+
+ Supplier> writerTasksProvider = () -> stageExecution.getAllTasks().stream()
+ .map(RemoteTask::getTaskStatus)
+ .collect(toList());
+
+ ScaledWriterScheduler scheduler = new ScaledWriterScheduler(
+ stageExecution,
+ sourceTasksProvider,
+ writerTasksProvider,
+ nodeScheduler.createNodeSelector(null),
+ scheduledExecutor,
+ getWriterMinSize(session));
+ whenAllStages(childStageExecutions, StageExecutionState::isDone)
+ .addListener(scheduler::finish, directExecutor());
+ return scheduler;
+ }
+ else {
+ if (!splitSources.isEmpty()) {
+ // contains local source
+ List schedulingOrder = plan.getFragment().getTableScanSchedulingOrder();
+ ConnectorId connectorId = partitioningHandle.getConnectorId().orElseThrow(IllegalStateException::new);
+ List connectorPartitionHandles;
+ boolean groupedExecutionForStage = plan.getFragment().getStageExecutionDescriptor().isStageGroupedExecution();
+ if (groupedExecutionForStage) {
+ connectorPartitionHandles = nodePartitioningManager.listPartitionHandles(session, partitioningHandle);
+ checkState(!ImmutableList.of(NOT_PARTITIONED).equals(connectorPartitionHandles));
+ }
+ else {
+ connectorPartitionHandles = ImmutableList.of(NOT_PARTITIONED);
+ }
+
+ BucketNodeMap bucketNodeMap;
+ List stageNodeList;
+ if (plan.getFragment().getRemoteSourceNodes().stream().allMatch(node -> node.getExchangeType() == REPLICATE)) {
+ // no non-replicated remote source
+ boolean dynamicLifespanSchedule = plan.getFragment().getStageExecutionDescriptor().isDynamicLifespanSchedule();
+ bucketNodeMap = nodePartitioningManager.getBucketNodeMap(session, partitioningHandle, dynamicLifespanSchedule);
+
+ // verify execution is consistent with planner's decision on dynamic lifespan schedule
+ verify(bucketNodeMap.isDynamic() == dynamicLifespanSchedule);
+
+ if (!bucketNodeMap.isDynamic()) {
+ stageNodeList = ((FixedBucketNodeMap) bucketNodeMap).getBucketToNode().stream()
+ .distinct()
+ .collect(toImmutableList());
+ }
+ else {
+ stageNodeList = new ArrayList<>(nodeScheduler.createNodeSelector(connectorId).selectRandomNodes(maxTasksPerStage));
+ }
+ }
+ else {
+ // cannot use dynamic lifespan schedule
+ verify(!plan.getFragment().getStageExecutionDescriptor().isDynamicLifespanSchedule());
+
+ // remote source requires nodePartitionMap
+ NodePartitionMap nodePartitionMap = partitioningCache.apply(plan.getFragment().getPartitioning());
+ if (groupedExecutionForStage) {
+ checkState(connectorPartitionHandles.size() == nodePartitionMap.getBucketToPartition().length);
+ }
+ stageNodeList = nodePartitionMap.getPartitionToNode();
+ bucketNodeMap = nodePartitionMap.asBucketNodeMap();
+ }
+
+ FixedSourcePartitionedScheduler fixedSourcePartitionedScheduler = new FixedSourcePartitionedScheduler(
+ stageExecution,
+ splitSources,
+ plan.getFragment().getStageExecutionDescriptor(),
+ schedulingOrder,
+ stageNodeList,
+ bucketNodeMap,
+ splitBatchSize,
+ getConcurrentLifespansPerNode(session),
+ nodeScheduler.createNodeSelector(connectorId),
+ connectorPartitionHandles);
+ if (plan.getFragment().getStageExecutionDescriptor().isRecoverableGroupedExecution()) {
+ stageExecution.registerStageTaskRecoveryCallback(taskId -> {
+ checkArgument(taskId.getStageExecutionId().getStageId().equals(stageId), "The task did not execute this stage");
+ checkArgument(parentStageExecution.isPresent(), "Parent stage execution must exist");
+ checkArgument(parentStageExecution.get().getAllTasks().size() == 1, "Parent stage should only have one task for recoverable grouped execution");
+
+ parentStageExecution.get().removeRemoteSourceIfSingleTaskStage(taskId);
+ fixedSourcePartitionedScheduler.recover(taskId);
+ });
+ }
+ return fixedSourcePartitionedScheduler;
+ }
+
+ else {
+ // all sources are remote
+ NodePartitionMap nodePartitionMap = partitioningCache.apply(plan.getFragment().getPartitioning());
+ List partitionToNode = nodePartitionMap.getPartitionToNode();
+ // todo this should asynchronously wait a standard timeout period before failing
+ checkCondition(!partitionToNode.isEmpty(), NO_NODES_AVAILABLE, "No worker nodes available");
+ return new FixedCountScheduler(stageExecution, partitionToNode);
+ }
+ }
+ }
+
+ private static Optional getBucketToPartition(
+ PartitioningHandle partitioningHandle,
+ Function partitioningCache,
+ PlanNode fragmentRoot,
+ List remoteSourceNodes)
+ {
+ if (partitioningHandle.equals(SOURCE_DISTRIBUTION) || partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
+ return Optional.of(new int[1]);
+ }
+ else if (PlanNodeSearcher.searchFrom(fragmentRoot).where(node -> node instanceof TableScanNode).findFirst().isPresent()) {
+ if (remoteSourceNodes.stream().allMatch(node -> node.getExchangeType() == REPLICATE)) {
+ return Optional.empty();
+ }
+ else {
+ // remote source requires nodePartitionMap
+ NodePartitionMap nodePartitionMap = partitioningCache.apply(partitioningHandle);
+ return Optional.of(nodePartitionMap.getBucketToPartition());
+ }
+ }
+ else {
+ NodePartitionMap nodePartitionMap = partitioningCache.apply(partitioningHandle);
+ List partitionToNode = nodePartitionMap.getPartitionToNode();
+ // todo this should asynchronously wait a standard timeout period before failing
+ checkCondition(!partitionToNode.isEmpty(), NO_NODES_AVAILABLE, "No worker nodes available");
+ return Optional.of(nodePartitionMap.getBucketToPartition());
+ }
+ }
+
+ private static ListenableFuture> whenAllStages(Collection stageExecutions, Predicate predicate)
+ {
+ checkArgument(!stageExecutions.isEmpty(), "stageExecutions is empty");
+ Set stageIds = newConcurrentHashSet(stageExecutions.stream()
+ .map(SqlStageExecution::getStageExecutionId)
+ .collect(toSet()));
+ SettableFuture> future = SettableFuture.create();
+
+ for (SqlStageExecution stage : stageExecutions) {
+ stage.addStateChangeListener(state -> {
+ if (predicate.test(state) && stageIds.remove(stage.getStageExecutionId()) && stageIds.isEmpty()) {
+ future.set(null);
+ }
+ });
+ }
+
+ return future;
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java
index 62f7e1590da5b..d29259d2e5707 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java
@@ -17,266 +17,172 @@
import com.facebook.airlift.stats.TimeStat;
import com.facebook.presto.Session;
import com.facebook.presto.execution.BasicStageExecutionStats;
+import com.facebook.presto.execution.ExecutionFailureInfo;
import com.facebook.presto.execution.LocationFactory;
-import com.facebook.presto.execution.NodeTaskMap;
import com.facebook.presto.execution.QueryState;
import com.facebook.presto.execution.QueryStateMachine;
import com.facebook.presto.execution.RemoteTask;
import com.facebook.presto.execution.RemoteTaskFactory;
import com.facebook.presto.execution.SqlStageExecution;
-import com.facebook.presto.execution.StageExecutionId;
import com.facebook.presto.execution.StageExecutionInfo;
import com.facebook.presto.execution.StageExecutionState;
import com.facebook.presto.execution.StageId;
import com.facebook.presto.execution.StageInfo;
import com.facebook.presto.execution.TaskId;
-import com.facebook.presto.execution.TaskStatus;
import com.facebook.presto.execution.buffer.OutputBuffers;
import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
-import com.facebook.presto.failureDetector.FailureDetector;
-import com.facebook.presto.metadata.InternalNode;
-import com.facebook.presto.metadata.Metadata;
-import com.facebook.presto.spi.ConnectorId;
+import com.facebook.presto.metadata.InternalNodeManager;
import com.facebook.presto.spi.PrestoException;
-import com.facebook.presto.spi.connector.ConnectorPartitionHandle;
-import com.facebook.presto.spi.plan.PlanNodeId;
-import com.facebook.presto.split.SplitSource;
-import com.facebook.presto.sql.planner.NodePartitionMap;
-import com.facebook.presto.sql.planner.NodePartitioningManager;
-import com.facebook.presto.sql.planner.PartitioningHandle;
import com.facebook.presto.sql.planner.PlanFragment;
import com.facebook.presto.sql.planner.SplitSourceFactory;
import com.facebook.presto.sql.planner.SubPlan;
import com.facebook.presto.sql.planner.plan.PlanFragmentId;
-import com.facebook.presto.sql.planner.plan.RemoteSourceNode;
+import com.google.common.base.VerifyException;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.primitives.Ints;
+import com.google.common.collect.ListMultimap;
import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
import io.airlift.units.Duration;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.function.Supplier;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
import static com.facebook.airlift.concurrent.MoreFutures.tryGetFutureValue;
import static com.facebook.airlift.concurrent.MoreFutures.whenAnyComplete;
import static com.facebook.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
-import static com.facebook.presto.SystemSessionProperties.getConcurrentLifespansPerNode;
import static com.facebook.presto.SystemSessionProperties.getMaxConcurrentMaterializations;
-import static com.facebook.presto.SystemSessionProperties.getMaxTasksPerStage;
-import static com.facebook.presto.SystemSessionProperties.getWriterMinSize;
+import static com.facebook.presto.SystemSessionProperties.getMaxStageRetries;
import static com.facebook.presto.execution.BasicStageExecutionStats.aggregateBasicStageStats;
-import static com.facebook.presto.execution.SqlStageExecution.createSqlStageExecution;
-import static com.facebook.presto.execution.StageExecutionState.ABORTED;
+import static com.facebook.presto.execution.SqlStageExecution.RECOVERABLE_ERROR_CODES;
+import static com.facebook.presto.execution.StageExecutionInfo.unscheduledExecutionInfo;
import static com.facebook.presto.execution.StageExecutionState.CANCELED;
import static com.facebook.presto.execution.StageExecutionState.FAILED;
import static com.facebook.presto.execution.StageExecutionState.FINISHED;
-import static com.facebook.presto.execution.StageExecutionState.PLANNED;
import static com.facebook.presto.execution.StageExecutionState.RUNNING;
import static com.facebook.presto.execution.StageExecutionState.SCHEDULED;
+import static com.facebook.presto.execution.buffer.OutputBuffers.BROADCAST_PARTITION_ID;
import static com.facebook.presto.execution.buffer.OutputBuffers.createDiscardingOutputBuffers;
-import static com.facebook.presto.execution.scheduler.SourcePartitionedScheduler.newSourcePartitionedSchedulerAsStageScheduler;
-import static com.facebook.presto.execution.scheduler.TableWriteInfo.createTableWriteInfo;
-import static com.facebook.presto.spi.ConnectorId.isInternalSystemConnector;
+import static com.facebook.presto.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers;
+import static com.facebook.presto.execution.scheduler.StreamingPlanSection.extractStreamingSections;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
-import static com.facebook.presto.spi.StandardErrorCode.NO_NODES_AVAILABLE;
-import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
-import static com.facebook.presto.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION;
-import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION;
-import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION;
-import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.REPLICATE;
-import static com.facebook.presto.util.Failures.checkCondition;
-import static com.google.common.base.Preconditions.checkArgument;
+import static com.facebook.presto.sql.planner.PlanFragmenter.ROOT_FRAGMENT_ID;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
-import static com.google.common.collect.ImmutableSet.toImmutableSet;
+import static com.google.common.collect.Iterables.getLast;
import static com.google.common.collect.Iterables.getOnlyElement;
-import static com.google.common.collect.Sets.newConcurrentHashSet;
import static com.google.common.collect.Streams.stream;
import static com.google.common.graph.Traverser.forTree;
-import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
-import static java.util.function.Function.identity;
-import static java.util.stream.Collectors.toList;
-import static java.util.stream.Collectors.toSet;
public class SqlQueryScheduler
+ implements SqlQuerySchedulerInterface
{
- private final QueryStateMachine queryStateMachine;
private final LocationFactory locationFactory;
private final ExecutionPolicy executionPolicy;
- private final SubPlan plan;
- private final StreamingPlanSection sectionedPlan;
- private final Map stageExecutions;
private final ExecutorService executor;
- private final StageId rootStageId;
private final SplitSchedulerStats schedulerStats;
+ private final SectionExecutionFactory sectionExecutionFactory;
+ private final RemoteTaskFactory remoteTaskFactory;
+ private final SplitSourceFactory splitSourceFactory;
+ private final InternalNodeManager nodeManager;
+
+ private final Session session;
+ private final QueryStateMachine queryStateMachine;
+ private final SubPlan plan;
+ private final StreamingPlanSection sectionedPlan;
private final boolean summarizeTaskInfo;
+ private final int maxConcurrentMaterializations;
+ private final int maxStageRetries;
+
+ private final Map> sectionExecutions = new ConcurrentHashMap<>();
private final AtomicBoolean started = new AtomicBoolean();
private final AtomicBoolean scheduling = new AtomicBoolean();
- private final int maxConcurrentMaterializations;
+ private final AtomicInteger retriedSections = new AtomicInteger();
public static SqlQueryScheduler createSqlQueryScheduler(
- QueryStateMachine queryStateMachine,
LocationFactory locationFactory,
- SubPlan plan,
- NodePartitioningManager nodePartitioningManager,
- NodeScheduler nodeScheduler,
+ ExecutionPolicy executionPolicy,
+ ExecutorService executor,
+ SplitSchedulerStats schedulerStats,
+ SectionExecutionFactory sectionExecutionFactory,
RemoteTaskFactory remoteTaskFactory,
SplitSourceFactory splitSourceFactory,
+ InternalNodeManager nodeManager,
Session session,
- boolean summarizeTaskInfo,
- int splitBatchSize,
- ExecutorService queryExecutor,
- ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- OutputBuffers rootOutputBuffers,
- NodeTaskMap nodeTaskMap,
- ExecutionPolicy executionPolicy,
- SplitSchedulerStats schedulerStats,
- Metadata metadata)
+ QueryStateMachine queryStateMachine,
+ SubPlan plan,
+ boolean summarizeTaskInfo)
{
SqlQueryScheduler sqlQueryScheduler = new SqlQueryScheduler(
- queryStateMachine,
locationFactory,
- plan,
- nodePartitioningManager,
- nodeScheduler,
+ executionPolicy,
+ executor,
+ schedulerStats,
+ sectionExecutionFactory,
remoteTaskFactory,
splitSourceFactory,
+ nodeManager,
session,
- summarizeTaskInfo,
- splitBatchSize,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- rootOutputBuffers,
- nodeTaskMap,
- executionPolicy,
- schedulerStats,
- metadata);
+ queryStateMachine,
+ plan,
+ summarizeTaskInfo);
sqlQueryScheduler.initialize();
return sqlQueryScheduler;
}
private SqlQueryScheduler(
- QueryStateMachine queryStateMachine,
LocationFactory locationFactory,
- SubPlan plan,
- NodePartitioningManager nodePartitioningManager,
- NodeScheduler nodeScheduler,
+ ExecutionPolicy executionPolicy,
+ ExecutorService executor,
+ SplitSchedulerStats schedulerStats,
+ SectionExecutionFactory sectionExecutionFactory,
RemoteTaskFactory remoteTaskFactory,
SplitSourceFactory splitSourceFactory,
+ InternalNodeManager nodeManager,
Session session,
- boolean summarizeTaskInfo,
- int splitBatchSize,
- ExecutorService queryExecutor,
- ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- OutputBuffers rootOutputBuffers,
- NodeTaskMap nodeTaskMap,
- ExecutionPolicy executionPolicy,
- SplitSchedulerStats schedulerStats,
- Metadata metadata)
+ QueryStateMachine queryStateMachine,
+ SubPlan plan,
+ boolean summarizeTaskInfo)
{
- this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null");
this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
- this.plan = requireNonNull(plan, "plan is null");
this.executionPolicy = requireNonNull(executionPolicy, "schedulerPolicyFactory is null");
+ this.executor = requireNonNull(executor, "executor is null");
this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null");
+ this.sectionExecutionFactory = requireNonNull(sectionExecutionFactory, "sectionExecutionFactory is null");
+ this.remoteTaskFactory = requireNonNull(remoteTaskFactory, "remoteTaskFactory is null");
+ this.splitSourceFactory = requireNonNull(splitSourceFactory, "splitSourceFactory is null");
+ this.nodeManager = requireNonNull(nodeManager, "nodeManager is null");
+ this.session = requireNonNull(session, "session is null");
+ this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null");
+ this.plan = requireNonNull(plan, "plan is null");
+ this.sectionedPlan = extractStreamingSections(plan);
this.summarizeTaskInfo = summarizeTaskInfo;
-
- OutputBufferId rootBufferId = getOnlyElement(rootOutputBuffers.getBuffers().keySet());
- sectionedPlan = extractStreamingSections(plan);
- List stageExecutions = createStageExecutions(
- (fragmentId, tasks, noMoreExchangeLocations) -> updateQueryOutputLocations(queryStateMachine, rootBufferId, tasks, noMoreExchangeLocations),
- sectionedPlan,
- Optional.of(new int[1]),
- metadata,
- rootOutputBuffers,
- nodeScheduler,
- remoteTaskFactory,
- splitSourceFactory,
- session,
- splitBatchSize,
- nodePartitioningManager,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- nodeTaskMap);
-
- this.rootStageId = Iterables.getLast(stageExecutions).getStageExecution().getStageExecutionId().getStageId();
-
- this.stageExecutions = stageExecutions.stream()
- .collect(toImmutableMap(execution -> execution.getStageExecution().getStageExecutionId().getStageId(), identity()));
-
- this.executor = queryExecutor;
this.maxConcurrentMaterializations = getMaxConcurrentMaterializations(session);
+ this.maxStageRetries = getMaxStageRetries(session);
}
// this is a separate method to ensure that the `this` reference is not leaked during construction
private void initialize()
{
- SqlStageExecution rootStage = stageExecutions.get(rootStageId).getStageExecution();
- rootStage.addStateChangeListener(state -> {
- if (state == FINISHED) {
- queryStateMachine.transitionToFinishing();
- }
- else if (state == CANCELED) {
- // output stage was canceled
- queryStateMachine.transitionToCanceled();
- }
- });
-
- for (StageExecutionAndScheduler stageExecutionInfo : stageExecutions.values()) {
- SqlStageExecution stageExecution = stageExecutionInfo.getStageExecution();
- stageExecution.addStateChangeListener(state -> {
- if (queryStateMachine.isDone()) {
- return;
- }
- if (state == FAILED) {
- queryStateMachine.transitionToFailed(stageExecution.getStageExecutionInfo().getFailureCause().get().toException());
- }
- else if (state == ABORTED) {
- // this should never happen, since abort can only be triggered in query clean up after the query is finished
- queryStateMachine.transitionToFailed(new PrestoException(GENERIC_INTERNAL_ERROR, "Query stage was aborted"));
- }
- else if (state == FINISHED) {
- // checks if there's any new sections available for execution and starts the scheduling if any
- startScheduling();
- }
- else if (queryStateMachine.getQueryState() == QueryState.STARTING) {
- // if the stage has at least one task, we are running
- if (stageExecution.hasTasks()) {
- queryStateMachine.transitionToRunning();
- }
- }
- });
- stageExecution.addFinalStageInfoListener(status -> queryStateMachine.updateQueryInfo(Optional.of(getStageInfo())));
- }
-
// when query is done or any time a stage completes, attempt to transition query to "final query info ready"
queryStateMachine.addStateChangeListener(newState -> {
if (newState.isDone()) {
@@ -285,399 +191,7 @@ else if (queryStateMachine.getQueryState() == QueryState.STARTING) {
});
}
- private static void updateQueryOutputLocations(QueryStateMachine queryStateMachine, OutputBufferId rootBufferId, Set tasks, boolean noMoreExchangeLocations)
- {
- Map bufferLocations = tasks.stream()
- .collect(toImmutableMap(
- task -> getBufferLocation(task, rootBufferId),
- RemoteTask::getTaskId));
- queryStateMachine.updateOutputLocations(bufferLocations, noMoreExchangeLocations);
- }
-
- private static URI getBufferLocation(RemoteTask remoteTask, OutputBufferId rootBufferId)
- {
- URI location = remoteTask.getTaskStatus().getSelf();
- return uriBuilderFrom(location).appendPath("results").appendPath(rootBufferId.toString()).build();
- }
-
- /**
- * returns a List of SqlStageExecutionInfos in a postorder representation of the tree
- */
- private List createStageExecutions(
- ExchangeLocationsConsumer locationsConsumer,
- StreamingPlanSection section,
- Optional bucketToPartition,
- Metadata metadata,
- OutputBuffers outputBuffers,
- NodeScheduler nodeScheduler,
- RemoteTaskFactory remoteTaskFactory,
- SplitSourceFactory splitSourceFactory,
- Session session,
- int splitBatchSize,
- NodePartitioningManager nodePartitioningManager,
- ExecutorService queryExecutor,
- ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- NodeTaskMap nodeTaskMap)
- {
- ImmutableList.Builder stages = ImmutableList.builder();
-
- for (StreamingPlanSection childSection : section.getChildren()) {
- stages.addAll(createStageExecutions(
- discardingLocationConsumer(),
- childSection,
- Optional.empty(),
- metadata,
- createDiscardingOutputBuffers(),
- nodeScheduler,
- remoteTaskFactory,
- splitSourceFactory,
- session,
- splitBatchSize,
- nodePartitioningManager,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- nodeTaskMap));
- }
-
- // Only fetch a distribution once per section to ensure all stages see the same machine assignments
- Map partitioningCache = new HashMap<>();
- TableWriteInfo tableWriteInfo = createTableWriteInfo(section.getPlan(), metadata, session);
- List sectionStages = createStreamingLinkedStageExecutions(
- locationsConsumer,
- section.getPlan().withBucketToPartition(bucketToPartition),
- nodeScheduler,
- remoteTaskFactory,
- splitSourceFactory,
- session,
- splitBatchSize,
- partitioningHandle -> partitioningCache.computeIfAbsent(partitioningHandle, handle -> nodePartitioningManager.getNodePartitioningMap(session, handle)),
- nodePartitioningManager,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- nodeTaskMap,
- tableWriteInfo,
- Optional.empty());
- Iterables.getLast(sectionStages)
- .getStageExecution()
- .setOutputBuffers(outputBuffers);
- stages.addAll(sectionStages);
-
- return stages.build();
- }
-
- /**
- * returns a List of SqlStageExecutionInfos in a postorder representation of the tree
- */
- private List createStreamingLinkedStageExecutions(
- ExchangeLocationsConsumer parent,
- StreamingSubPlan plan,
- NodeScheduler nodeScheduler,
- RemoteTaskFactory remoteTaskFactory,
- SplitSourceFactory splitSourceFactory,
- Session session,
- int splitBatchSize,
- Function partitioningCache,
- NodePartitioningManager nodePartitioningManager,
- ExecutorService queryExecutor,
- ScheduledExecutorService schedulerExecutor,
- FailureDetector failureDetector,
- NodeTaskMap nodeTaskMap,
- TableWriteInfo tableWriteInfo,
- Optional parentStageExecution)
- {
- ImmutableList.Builder stageExecutionInfos = ImmutableList.builder();
-
- PlanFragmentId fragmentId = plan.getFragment().getId();
- StageId stageId = getStageId(fragmentId);
- SqlStageExecution stageExecution = createSqlStageExecution(
- new StageExecutionId(stageId, 0),
- plan.getFragment(),
- remoteTaskFactory,
- session,
- summarizeTaskInfo,
- nodeTaskMap,
- queryExecutor,
- failureDetector,
- schedulerStats,
- tableWriteInfo);
-
- PartitioningHandle partitioningHandle = plan.getFragment().getPartitioning();
- Map splitSources = splitSourceFactory.createSplitSources(plan.getFragment(), session, tableWriteInfo);
- List remoteSourceNodes = plan.getFragment().getRemoteSourceNodes();
- Optional bucketToPartition = getBucketToPartition(partitioningHandle, partitioningCache, splitSources, remoteSourceNodes);
-
- // create child stages
- ImmutableSet.Builder childStagesBuilder = ImmutableSet.builder();
- for (StreamingSubPlan stagePlan : plan.getChildren()) {
- List subTree = createStreamingLinkedStageExecutions(
- stageExecution::addExchangeLocations,
- stagePlan.withBucketToPartition(bucketToPartition),
- nodeScheduler,
- remoteTaskFactory,
- splitSourceFactory,
- session,
- splitBatchSize,
- partitioningCache,
- nodePartitioningManager,
- queryExecutor,
- schedulerExecutor,
- failureDetector,
- nodeTaskMap,
- tableWriteInfo,
- Optional.of(stageExecution));
- stageExecutionInfos.addAll(subTree);
- childStagesBuilder.add(Iterables.getLast(subTree).getStageExecution());
- }
- Set childStageExecutions = childStagesBuilder.build();
- stageExecution.addStateChangeListener(newState -> {
- if (newState.isDone()) {
- childStageExecutions.forEach(SqlStageExecution::cancel);
- }
- });
-
- StageScheduler stageScheduler = createStageScheduler(
- plan,
- nodeScheduler,
- session,
- splitBatchSize,
- partitioningCache,
- nodePartitioningManager,
- schedulerExecutor,
- parentStageExecution,
- stageId,
- stageExecution,
- partitioningHandle,
- splitSources,
- childStageExecutions);
- StageLinkage stageLinkage = new StageLinkage(fragmentId, parent, childStageExecutions);
- stageExecutionInfos.add(new StageExecutionAndScheduler(stageExecution, stageLinkage, stageScheduler));
-
- return stageExecutionInfos.build();
- }
-
- private StageScheduler createStageScheduler(
- StreamingSubPlan plan,
- NodeScheduler nodeScheduler,
- Session session,
- int splitBatchSize,
- Function partitioningCache,
- NodePartitioningManager nodePartitioningManager,
- ScheduledExecutorService schedulerExecutor,
- Optional parentStageExecution,
- StageId stageId, SqlStageExecution stageExecution,
- PartitioningHandle partitioningHandle,
- Map splitSources,
- Set childStageExecutions)
- {
- int maxTasksPerStage = getMaxTasksPerStage(session);
- if (partitioningHandle.equals(SOURCE_DISTRIBUTION)) {
- // TODO: defer opening split sources when stage scheduling starts
- // nodes are selected dynamically based on the constraints of the splits and the system load
- Entry entry = getOnlyElement(splitSources.entrySet());
- PlanNodeId planNodeId = entry.getKey();
- SplitSource splitSource = entry.getValue();
- ConnectorId connectorId = splitSource.getConnectorId();
- if (isInternalSystemConnector(connectorId)) {
- connectorId = null;
- }
-
- NodeSelector nodeSelector = nodeScheduler.createNodeSelector(connectorId, maxTasksPerStage);
- SplitPlacementPolicy placementPolicy = new DynamicSplitPlacementPolicy(nodeSelector, stageExecution::getAllTasks);
-
- checkArgument(!plan.getFragment().getStageExecutionDescriptor().isStageGroupedExecution());
- return newSourcePartitionedSchedulerAsStageScheduler(stageExecution, planNodeId, splitSource, placementPolicy, splitBatchSize);
- }
- else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
- Supplier> sourceTasksProvider = () -> childStageExecutions.stream()
- .map(SqlStageExecution::getAllTasks)
- .flatMap(Collection::stream)
- .map(RemoteTask::getTaskStatus)
- .collect(toList());
-
- Supplier> writerTasksProvider = () -> stageExecution.getAllTasks().stream()
- .map(RemoteTask::getTaskStatus)
- .collect(toList());
-
- ScaledWriterScheduler scheduler = new ScaledWriterScheduler(
- stageExecution,
- sourceTasksProvider,
- writerTasksProvider,
- nodeScheduler.createNodeSelector(null),
- schedulerExecutor,
- getWriterMinSize(session));
- whenAllStages(childStageExecutions, StageExecutionState::isDone)
- .addListener(scheduler::finish, directExecutor());
- return scheduler;
- }
- else {
- // TODO: defer opening split sources when stage scheduling starts
- if (!splitSources.isEmpty()) {
- // contains local source
- List schedulingOrder = plan.getFragment().getTableScanSchedulingOrder();
- ConnectorId connectorId = partitioningHandle.getConnectorId().orElseThrow(IllegalStateException::new);
- List connectorPartitionHandles;
- boolean groupedExecutionForStage = plan.getFragment().getStageExecutionDescriptor().isStageGroupedExecution();
- if (groupedExecutionForStage) {
- connectorPartitionHandles = nodePartitioningManager.listPartitionHandles(session, partitioningHandle);
- checkState(!ImmutableList.of(NOT_PARTITIONED).equals(connectorPartitionHandles));
- }
- else {
- connectorPartitionHandles = ImmutableList.of(NOT_PARTITIONED);
- }
-
- BucketNodeMap bucketNodeMap;
- List stageNodeList;
- if (plan.getFragment().getRemoteSourceNodes().stream().allMatch(node -> node.getExchangeType() == REPLICATE)) {
- // no non-replicated remote source
- boolean dynamicLifespanSchedule = plan.getFragment().getStageExecutionDescriptor().isDynamicLifespanSchedule();
- bucketNodeMap = nodePartitioningManager.getBucketNodeMap(session, partitioningHandle, dynamicLifespanSchedule);
-
- // verify execution is consistent with planner's decision on dynamic lifespan schedule
- verify(bucketNodeMap.isDynamic() == dynamicLifespanSchedule);
-
- if (!bucketNodeMap.isDynamic()) {
- stageNodeList = ((FixedBucketNodeMap) bucketNodeMap).getBucketToNode().stream()
- .distinct()
- .collect(toImmutableList());
- }
- else {
- stageNodeList = new ArrayList<>(nodeScheduler.createNodeSelector(connectorId).selectRandomNodes(maxTasksPerStage));
- }
- }
- else {
- // cannot use dynamic lifespan schedule
- verify(!plan.getFragment().getStageExecutionDescriptor().isDynamicLifespanSchedule());
-
- // remote source requires nodePartitionMap
- NodePartitionMap nodePartitionMap = partitioningCache.apply(plan.getFragment().getPartitioning());
- if (groupedExecutionForStage) {
- checkState(connectorPartitionHandles.size() == nodePartitionMap.getBucketToPartition().length);
- }
- stageNodeList = nodePartitionMap.getPartitionToNode();
- bucketNodeMap = nodePartitionMap.asBucketNodeMap();
- }
-
- FixedSourcePartitionedScheduler fixedSourcePartitionedScheduler = new FixedSourcePartitionedScheduler(
- stageExecution,
- splitSources,
- plan.getFragment().getStageExecutionDescriptor(),
- schedulingOrder,
- stageNodeList,
- bucketNodeMap,
- splitBatchSize,
- getConcurrentLifespansPerNode(session),
- nodeScheduler.createNodeSelector(connectorId),
- connectorPartitionHandles);
- if (plan.getFragment().getStageExecutionDescriptor().isRecoverableGroupedExecution()) {
- stageExecution.registerStageTaskRecoveryCallback(taskId -> {
- checkArgument(taskId.getStageExecutionId().getStageId().equals(stageId), "The task did not execute this stage");
- checkArgument(parentStageExecution.isPresent(), "Parent stage execution must exist");
- checkArgument(parentStageExecution.get().getAllTasks().size() == 1, "Parent stage should only have one task for recoverable grouped execution");
-
- parentStageExecution.get().removeRemoteSourceIfSingleTaskStage(taskId);
- fixedSourcePartitionedScheduler.recover(taskId);
- });
- }
- return fixedSourcePartitionedScheduler;
- }
-
- else {
- // all sources are remote
- NodePartitionMap nodePartitionMap = partitioningCache.apply(plan.getFragment().getPartitioning());
- List partitionToNode = nodePartitionMap.getPartitionToNode();
- // todo this should asynchronously wait a standard timeout period before failing
- checkCondition(!partitionToNode.isEmpty(), NO_NODES_AVAILABLE, "No worker nodes available");
- return new FixedCountScheduler(stageExecution, partitionToNode);
- }
- }
- }
-
- private Optional getBucketToPartition(
- PartitioningHandle partitioningHandle,
- Function partitioningCache,
- Map splitSources,
- List remoteSourceNodes)
- {
- if (partitioningHandle.equals(SOURCE_DISTRIBUTION) || partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
- return Optional.of(new int[1]);
- }
- else if (!splitSources.isEmpty()) {
- if (remoteSourceNodes.stream().allMatch(node -> node.getExchangeType() == REPLICATE)) {
- return Optional.empty();
- }
- else {
- // remote source requires nodePartitionMap
- NodePartitionMap nodePartitionMap = partitioningCache.apply(partitioningHandle);
- return Optional.of(nodePartitionMap.getBucketToPartition());
- }
- }
- else {
- NodePartitionMap nodePartitionMap = partitioningCache.apply(partitioningHandle);
- List partitionToNode = nodePartitionMap.getPartitionToNode();
- // todo this should asynchronously wait a standard timeout period before failing
- checkCondition(!partitionToNode.isEmpty(), NO_NODES_AVAILABLE, "No worker nodes available");
- return Optional.of(nodePartitionMap.getBucketToPartition());
- }
- }
-
- public BasicStageExecutionStats getBasicStageStats()
- {
- List stageStats = stageExecutions.values().stream()
- .map(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getBasicStageStats())
- .collect(toImmutableList());
-
- return aggregateBasicStageStats(stageStats);
- }
-
- public StageInfo getStageInfo()
- {
- Map stageInfos = stageExecutions.values().stream()
- .map(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getStageExecutionInfo())
- .collect(toImmutableMap(execution -> execution.getStageExecutionId().getStageId(), identity()));
-
- return buildStageInfo(plan, stageInfos);
- }
-
- private StageInfo buildStageInfo(SubPlan subPlan, Map stageExecutionInfos)
- {
- StageId stageId = getStageId(subPlan.getFragment().getId());
- StageExecutionInfo stageExecutionInfo = stageExecutionInfos.get(stageId);
- checkArgument(stageExecutionInfo != null, "No stageExecutionInfo for %s", stageId);
- return new StageInfo(
- stageId,
- locationFactory.createStageLocation(stageId),
- Optional.of(subPlan.getFragment()),
- stageExecutionInfo,
- ImmutableList.of(),
- subPlan.getChildren().stream()
- .map(plan -> buildStageInfo(plan, stageExecutionInfos))
- .collect(toImmutableList()));
- }
-
- public long getUserMemoryReservation()
- {
- return stageExecutions.values().stream()
- .mapToLong(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getUserMemoryReservation())
- .sum();
- }
-
- public long getTotalMemoryReservation()
- {
- return stageExecutions.values().stream()
- .mapToLong(stageExecutionInfo -> stageExecutionInfo.getStageExecution().getTotalMemoryReservation())
- .sum();
- }
-
- public Duration getTotalCpuTime()
- {
- long millis = stageExecutions.values().stream()
- .mapToLong(stage -> stage.getStageExecution().getTotalCpuTime().toMillis())
- .sum();
- return new Duration(millis, MILLISECONDS);
- }
+ @Override
public void start()
{
@@ -688,7 +202,6 @@ public void start()
private void startScheduling()
{
- requireNonNull(stageExecutions);
// still scheduling the previous batch of stages
if (scheduling.get()) {
return;
@@ -708,53 +221,54 @@ private void schedule()
try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
Set completedStages = new HashSet<>();
- List sectionExecutionSchedules = new LinkedList<>();
+ List executionSchedules = new LinkedList<>();
while (!Thread.currentThread().isInterrupted()) {
// remove finished section
- sectionExecutionSchedules.removeIf(ExecutionSchedule::isFinished);
+ executionSchedules.removeIf(ExecutionSchedule::isFinished);
// try to pull more section that are ready to be run
List sectionsReadyForExecution = getSectionsReadyForExecution();
// all finished
- if (sectionsReadyForExecution.isEmpty() && sectionExecutionSchedules.isEmpty()) {
+ if (sectionsReadyForExecution.isEmpty() && executionSchedules.isEmpty()) {
break;
}
- List> sectionStageExecutions = getStageExecutions(sectionsReadyForExecution);
- sectionStageExecutions.forEach(scheduledStageExecutions::addAll);
- sectionStageExecutions.stream()
- .map(executionInfos -> executionInfos.stream()
- .map(StageExecutionAndScheduler::getStageExecution)
- .collect(toImmutableList()))
+ List sectionExecutions = createStageExecutions(sectionsReadyForExecution);
+ if (queryStateMachine.isDone()) {
+ sectionExecutions.forEach(SectionExecution::abort);
+ }
+
+ sectionExecutions.forEach(sectionExecution -> scheduledStageExecutions.addAll(sectionExecution.getSectionStages()));
+ sectionExecutions.stream()
+ .map(SectionExecution::getSectionStages)
.map(executionPolicy::createExecutionSchedule)
- .forEach(sectionExecutionSchedules::add);
+ .forEach(executionSchedules::add);
- while (sectionExecutionSchedules.stream().noneMatch(ExecutionSchedule::isFinished)) {
+ while (!executionSchedules.isEmpty() && executionSchedules.stream().noneMatch(ExecutionSchedule::isFinished)) {
List> blockedStages = new ArrayList<>();
- List executionsToSchedule = sectionExecutionSchedules.stream()
+ List executionsToSchedule = executionSchedules.stream()
.flatMap(schedule -> schedule.getStagesToSchedule().stream())
.collect(toImmutableList());
- for (SqlStageExecution stageExecution : executionsToSchedule) {
- StageId stageId = stageExecution.getStageExecutionId().getStageId();
- stageExecution.beginScheduling();
+ for (StageExecutionAndScheduler executionAndScheduler : executionsToSchedule) {
+ executionAndScheduler.getStageExecution().beginScheduling();
// perform some scheduling work
- ScheduleResult result = stageExecutions.get(stageId).getStageScheduler()
+ ScheduleResult result = executionAndScheduler.getStageScheduler()
.schedule();
// modify parent and children based on the results of the scheduling
if (result.isFinished()) {
- stageExecution.schedulingComplete();
+ executionAndScheduler.getStageExecution().schedulingComplete();
}
else if (!result.getBlocked().isDone()) {
blockedStages.add(result.getBlocked());
}
- stageExecutions.get(stageId).getStageLinkage()
- .processScheduleResults(stageExecution.getState(), result.getNewTasks());
+ executionAndScheduler.getStageLinkage()
+ .processScheduleResults(executionAndScheduler.getStageExecution().getState(), result.getNewTasks());
schedulerStats.getSplitsScheduledPerIteration().add(result.getSplitsScheduled());
if (result.getBlockedReason().isPresent()) {
switch (result.getBlockedReason().get()) {
@@ -781,11 +295,11 @@ else if (!result.getBlocked().isDone()) {
// make sure to update stage linkage at least once per loop to catch async state changes (e.g., partial cancel)
boolean stageFinishedExecution = false;
- for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
- SqlStageExecution stageExecution = stageExecutionInfo.getStageExecution();
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : scheduledStageExecutions) {
+ SqlStageExecution stageExecution = stageExecutionAndScheduler.getStageExecution();
StageId stageId = stageExecution.getStageExecutionId().getStageId();
if (!completedStages.contains(stageId) && stageExecution.getState().isDone()) {
- stageExecutionInfo.getStageLinkage()
+ stageExecutionAndScheduler.getStageLinkage()
.processScheduleResults(stageExecution.getState(), ImmutableSet.of());
completedStages.add(stageId);
stageFinishedExecution = true;
@@ -809,10 +323,10 @@ else if (!result.getBlocked().isDone()) {
}
}
- for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
- StageExecutionState state = stageExecutionInfo.getStageExecution().getState();
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : scheduledStageExecutions) {
+ StageExecutionState state = stageExecutionAndScheduler.getStageExecution().getState();
if (state != SCHEDULED && state != RUNNING && !state.isDone()) {
- throw new PrestoException(GENERIC_INTERNAL_ERROR, format("Scheduling is complete, but stage execution %s is in state %s", stageExecutionInfo.getStageExecution().getStageExecutionId(), state));
+ throw new PrestoException(GENERIC_INTERNAL_ERROR, format("Scheduling is complete, but stage execution %s is in state %s", stageExecutionAndScheduler.getStageExecution().getStageExecutionId(), state));
}
}
@@ -829,9 +343,9 @@ else if (!result.getBlocked().isDone()) {
}
finally {
RuntimeException closeError = new RuntimeException();
- for (StageExecutionAndScheduler stageExecutionInfo : scheduledStageExecutions) {
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : scheduledStageExecutions) {
try {
- stageExecutionInfo.getStageScheduler().close();
+ stageExecutionAndScheduler.getStageScheduler().close();
}
catch (Throwable t) {
queryStateMachine.transitionToFailed(t);
@@ -851,8 +365,10 @@ private List getSectionsReadyForExecution()
{
long runningPlanSections =
stream(forTree(StreamingPlanSection::getChildren).depthFirstPreOrder(sectionedPlan))
- .map(section -> getStageExecution(section.getPlan().getFragment().getId()).getState())
- .filter(state -> !state.isDone() && state != PLANNED)
+ .map(section -> getLatestSectionExecution(getStageId(section.getPlan().getFragment().getId())))
+ .filter(Optional::isPresent)
+ .map(Optional::get)
+ .filter(SectionExecution::isRunning)
.count();
return stream(forTree(StreamingPlanSection::getChildren).depthFirstPreOrder(sectionedPlan))
// get all sections ready for execution
@@ -863,267 +379,275 @@ private List getSectionsReadyForExecution()
private boolean isReadyForExecution(StreamingPlanSection section)
{
- SqlStageExecution stageExecution = getStageExecution(section.getPlan().getFragment().getId());
- if (stageExecution.getState() != PLANNED) {
+ Optional sectionExecution = getLatestSectionExecution(getStageId(section.getPlan().getFragment().getId()));
+ if (sectionExecution.isPresent() && (sectionExecution.get().isRunning() || sectionExecution.get().isFinished())) {
// already scheduled
return false;
}
for (StreamingPlanSection child : section.getChildren()) {
- SqlStageExecution rootStageExecution = getStageExecution(child.getPlan().getFragment().getId());
- if (rootStageExecution.getState() != FINISHED) {
+ Optional childSectionExecution = getLatestSectionExecution(getStageId(child.getPlan().getFragment().getId()));
+ if (!childSectionExecution.isPresent() || !childSectionExecution.get().isFinished()) {
return false;
}
}
return true;
}
- private List> getStageExecutions(List sections)
+ private Optional getLatestSectionExecution(StageId stageId)
{
- return sections.stream()
- .map(section -> stream(forTree(StreamingSubPlan::getChildren).depthFirstPreOrder(section.getPlan())).collect(toImmutableList()))
- .map(plans -> plans.stream()
- .map(StreamingSubPlan::getFragment)
- .map(PlanFragment::getId)
- .map(this::getStageExecutionInfo)
- .collect(toImmutableList()))
- .collect(toImmutableList());
+ List sectionExecutions = this.sectionExecutions.get(stageId);
+ if (sectionExecutions == null || sectionExecutions.isEmpty()) {
+ return Optional.empty();
+ }
+ return Optional.of(getLast(sectionExecutions));
}
- private SqlStageExecution getStageExecution(PlanFragmentId planFragmentId)
+ private StageId getStageId(PlanFragmentId fragmentId)
{
- return stageExecutions.get(getStageId(planFragmentId)).getStageExecution();
+ return new StageId(session.getQueryId(), fragmentId.getId());
}
- private StageExecutionAndScheduler getStageExecutionInfo(PlanFragmentId planFragmentId)
+ private List createStageExecutions(List sections)
{
- return stageExecutions.get(getStageId(planFragmentId));
- }
+ ImmutableList.Builder result = ImmutableList.builder();
+ for (StreamingPlanSection section : sections) {
+ StageId sectionId = getStageId(section.getPlan().getFragment().getId());
+ List attempts = sectionExecutions.computeIfAbsent(sectionId, (ignored) -> new CopyOnWriteArrayList<>());
- private StageId getStageId(PlanFragmentId fragmentId)
- {
- return new StageId(queryStateMachine.getQueryId(), fragmentId.getId());
+ // sectionExecutions only get created when they are about to be scheduled, so there should
+ // never be a non-failed SectionExecution for a section that's ready for execution
+ verify(attempts.isEmpty() || getLast(attempts).isFailed(), "Non-failed sectionExecutions already exists");
+
+ PlanFragment sectionRootFragment = section.getPlan().getFragment();
+
+ Optional bucketToPartition;
+ OutputBuffers outputBuffers;
+ ExchangeLocationsConsumer locationsConsumer;
+ if (isRootFragment(sectionRootFragment)) {
+ bucketToPartition = Optional.of(new int[1]);
+ outputBuffers = createInitialEmptyOutputBuffers(sectionRootFragment.getPartitioningScheme().getPartitioning().getHandle())
+ .withBuffer(new OutputBufferId(0), BROADCAST_PARTITION_ID)
+ .withNoMoreBufferIds();
+ OutputBufferId rootBufferId = getOnlyElement(outputBuffers.getBuffers().keySet());
+ locationsConsumer = (fragmentId, tasks, noMoreExchangeLocations) ->
+ updateQueryOutputLocations(queryStateMachine, rootBufferId, tasks, noMoreExchangeLocations);
+ }
+ else {
+ bucketToPartition = Optional.empty();
+ outputBuffers = createDiscardingOutputBuffers();
+ locationsConsumer = (fragmentId, tasks, noMoreExchangeLocations) -> {};
+ }
+
+ int attemptId = attempts.size();
+ SectionExecution sectionExecution = sectionExecutionFactory.createSectionExecutions(
+ session,
+ section,
+ locationsConsumer,
+ bucketToPartition,
+ outputBuffers,
+ summarizeTaskInfo,
+ remoteTaskFactory,
+ splitSourceFactory,
+ attemptId);
+
+ addStateChangeListeners(sectionExecution);
+ attempts.add(sectionExecution);
+ result.add(sectionExecution);
+ }
+ return result.build();
}
- public void cancelStage(StageId stageId)
+ private static void updateQueryOutputLocations(QueryStateMachine queryStateMachine, OutputBufferId rootBufferId, Set tasks, boolean noMoreExchangeLocations)
{
- try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
- SqlStageExecution execution = stageExecutions.get(stageId).getStageExecution();
- SqlStageExecution stage = requireNonNull(execution, () -> format("Stage %s does not exist", stageId));
- stage.cancel();
- }
+ Map bufferLocations = tasks.stream()
+ .collect(toImmutableMap(
+ task -> getBufferLocation(task, rootBufferId),
+ RemoteTask::getTaskId));
+ queryStateMachine.updateOutputLocations(bufferLocations, noMoreExchangeLocations);
}
- public void abort()
+ private static URI getBufferLocation(RemoteTask remoteTask, OutputBufferId rootBufferId)
{
- try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
- stageExecutions.values().forEach(stageExecutionInfo -> stageExecutionInfo.getStageExecution().abort());
- }
+ URI location = remoteTask.getTaskStatus().getSelf();
+ return uriBuilderFrom(location).appendPath("results").appendPath(rootBufferId.toString()).build();
}
- private static ListenableFuture> whenAllStages(Collection stageExecutions, Predicate predicate)
+ private void addStateChangeListeners(SectionExecution sectionExecution)
{
- checkArgument(!stageExecutions.isEmpty(), "stageExecutions is empty");
- Set stageIds = newConcurrentHashSet(stageExecutions.stream()
- .map(SqlStageExecution::getStageExecutionId)
- .collect(toSet()));
- SettableFuture> future = SettableFuture.create();
+ for (StageExecutionAndScheduler stageExecutionAndScheduler : sectionExecution.getSectionStages()) {
+ SqlStageExecution stageExecution = stageExecutionAndScheduler.getStageExecution();
+ if (isRootFragment(stageExecution.getFragment())) {
+ stageExecution.addStateChangeListener(state -> {
+ if (state == FINISHED) {
+ queryStateMachine.transitionToFinishing();
+ }
+ else if (state == CANCELED) {
+ // output stage was canceled
+ queryStateMachine.transitionToCanceled();
+ }
+ });
+ }
+ stageExecution.addStateChangeListener(state -> {
+ if (queryStateMachine.isDone()) {
+ return;
+ }
+
+ if (state == FAILED) {
+ ExecutionFailureInfo failureInfo = stageExecution.getStageExecutionInfo().getFailureCause()
+ .orElseThrow(() -> new VerifyException(format("stage execution failed, but the failure info is missing: %s", stageExecution.getStageExecutionId())));
+ Exception failureException = failureInfo.toException();
+
+ boolean isRootSection = isRootFragment(sectionExecution.getRootStage().getStageExecution().getFragment());
+ // root section directly streams the results to the user, cannot be retried
+ if (isRootSection) {
+ queryStateMachine.transitionToFailed(failureException);
+ return;
+ }
+
+ if (retriedSections.get() >= maxStageRetries) {
+ queryStateMachine.transitionToFailed(failureException);
+ return;
+ }
- for (SqlStageExecution stage : stageExecutions) {
- stage.addStateChangeListener(state -> {
- if (predicate.test(state) && stageIds.remove(stage.getStageExecutionId()) && stageIds.isEmpty()) {
- future.set(null);
+ if (!RECOVERABLE_ERROR_CODES.contains(failureInfo.getErrorCode())) {
+ queryStateMachine.transitionToFailed(failureException);
+ return;
+ }
+
+ try {
+ if (sectionExecution.abort()) {
+ retriedSections.incrementAndGet();
+ nodeManager.refreshNodes();
+ startScheduling();
+ }
+ }
+ catch (Throwable t) {
+ if (failureException != t) {
+ failureException.addSuppressed(t);
+ }
+ queryStateMachine.transitionToFailed(failureException);
+ }
+ }
+ else if (state == FINISHED) {
+ // checks if there's any new sections available for execution and starts the scheduling if any
+ startScheduling();
+ }
+ else if (queryStateMachine.getQueryState() == QueryState.STARTING) {
+ // if the stage has at least one task, we are running
+ if (stageExecution.hasTasks()) {
+ queryStateMachine.transitionToRunning();
+ }
}
});
+ stageExecution.addFinalStageInfoListener(status -> queryStateMachine.updateQueryInfo(Optional.of(getStageInfo())));
}
-
- return future;
}
- public static StreamingPlanSection extractStreamingSections(SubPlan subPlan)
+ private static boolean isRootFragment(PlanFragment fragment)
{
- ImmutableList.Builder materializedExchangeChildren = ImmutableList.builder();
- StreamingSubPlan streamingSection = extractStreamingSection(subPlan, materializedExchangeChildren);
- return new StreamingPlanSection(
- streamingSection,
- materializedExchangeChildren.build().stream()
- .map(SqlQueryScheduler::extractStreamingSections)
- .collect(toImmutableList()));
+ return fragment.getId().getId() == ROOT_FRAGMENT_ID;
}
- private static StreamingSubPlan extractStreamingSection(SubPlan subPlan, ImmutableList.Builder materializedExchangeChildren)
+ @Override
+ public long getUserMemoryReservation()
{
- ImmutableList.Builder streamingSources = ImmutableList.builder();
- Set streamingFragmentIds = subPlan.getFragment().getRemoteSourceNodes().stream()
- .map(RemoteSourceNode::getSourceFragmentIds)
- .flatMap(List::stream)
- .collect(toImmutableSet());
- for (SubPlan child : subPlan.getChildren()) {
- if (streamingFragmentIds.contains(child.getFragment().getId())) {
- streamingSources.add(extractStreamingSection(child, materializedExchangeChildren));
- }
- else {
- materializedExchangeChildren.add(child);
- }
- }
- return new StreamingSubPlan(subPlan.getFragment(), streamingSources.build());
+ return getAllStagesExecutions().mapToLong(SqlStageExecution::getUserMemoryReservation).sum();
}
- private interface ExchangeLocationsConsumer
+ @Override
+ public long getTotalMemoryReservation()
{
- void addExchangeLocations(PlanFragmentId fragmentId, Set tasks, boolean noMoreExchangeLocations);
+ return getAllStagesExecutions().mapToLong(SqlStageExecution::getTotalMemoryReservation).sum();
}
- private static ExchangeLocationsConsumer discardingLocationConsumer()
+ @Override
+ public Duration getTotalCpuTime()
{
- return (fragmentId, tasks, noMoreExchangeLocations) -> {};
+ long millis = getAllStagesExecutions()
+ .map(SqlStageExecution::getTotalCpuTime)
+ .mapToLong(Duration::toMillis)
+ .sum();
+ return new Duration(millis, MILLISECONDS);
}
- private static class StageLinkage
+ @Override
+ public BasicStageExecutionStats getBasicStageStats()
{
- private final PlanFragmentId currentStageFragmentId;
- private final ExchangeLocationsConsumer parent;
- private final Set childOutputBufferManagers;
-
- public StageLinkage(PlanFragmentId fragmentId, ExchangeLocationsConsumer parent, Set children)
- {
- this.currentStageFragmentId = fragmentId;
- this.parent = parent;
- this.childOutputBufferManagers = children.stream()
- .map(childStage -> {
- PartitioningHandle partitioningHandle = childStage.getFragment().getPartitioningScheme().getPartitioning().getHandle();
- if (partitioningHandle.equals(FIXED_BROADCAST_DISTRIBUTION)) {
- return new BroadcastOutputBufferManager(childStage::setOutputBuffers);
- }
- else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
- return new ScaledOutputBufferManager(childStage::setOutputBuffers);
- }
- else {
- int partitionCount = Ints.max(childStage.getFragment().getPartitioningScheme().getBucketToPartition().get()) + 1;
- return new PartitionedOutputBufferManager(partitioningHandle, partitionCount, childStage::setOutputBuffers);
- }
- })
- .collect(toImmutableSet());
- }
-
- public void processScheduleResults(StageExecutionState newState, Set newTasks)
- {
- boolean noMoreTasks = false;
- switch (newState) {
- case PLANNED:
- case SCHEDULING:
- // workers are still being added to the query
- break;
- case FINISHED_TASK_SCHEDULING:
- case SCHEDULING_SPLITS:
- case SCHEDULED:
- case RUNNING:
- case FINISHED:
- case CANCELED:
- // no more workers will be added to the query
- noMoreTasks = true;
- case ABORTED:
- case FAILED:
- // DO NOT complete a FAILED or ABORTED stage. This will cause the
- // stage above to finish normally, which will result in a query
- // completing successfully when it should fail..
- break;
- }
-
- // Add an exchange location to the parent stage for each new task
- parent.addExchangeLocations(currentStageFragmentId, newTasks, noMoreTasks);
-
- if (!childOutputBufferManagers.isEmpty()) {
- // Add an output buffer to the child stages for each new task
- List newOutputBuffers = newTasks.stream()
- .map(task -> new OutputBufferId(task.getTaskId().getId()))
- .collect(toImmutableList());
- for (OutputBufferManager child : childOutputBufferManagers) {
- child.addOutputBuffers(newOutputBuffers, noMoreTasks);
- }
- }
- }
+ List stageStats = getAllStagesExecutions()
+ .map(SqlStageExecution::getBasicStageStats)
+ .collect(toImmutableList());
+ return aggregateBasicStageStats(stageStats);
}
- public static class StreamingPlanSection
+ private Stream getAllStagesExecutions()
{
- private final StreamingSubPlan plan;
- // materialized exchange children
- private final List children;
-
- public StreamingPlanSection(StreamingSubPlan plan, List children)
- {
- this.plan = requireNonNull(plan, "plan is null");
- this.children = ImmutableList.copyOf(requireNonNull(children, "children is null"));
- }
-
- public StreamingSubPlan getPlan()
- {
- return plan;
- }
-
- public List getChildren()
- {
- return children;
- }
+ return sectionExecutions.values().stream()
+ .flatMap(Collection::stream)
+ .flatMap(sectionExecution -> sectionExecution.getSectionStages().stream())
+ .map(StageExecutionAndScheduler::getStageExecution);
}
- /**
- * StreamingSubPlan is similar to SubPlan but only contains streaming children
- */
- public static class StreamingSubPlan
+ @Override
+ public StageInfo getStageInfo()
{
- private final PlanFragment fragment;
- // streaming children
- private final List children;
-
- public StreamingSubPlan(PlanFragment fragment, List children)
- {
- this.fragment = requireNonNull(fragment, "fragment is null");
- this.children = ImmutableList.copyOf(requireNonNull(children, "children is null"));
- }
-
- public PlanFragment getFragment()
- {
- return fragment;
- }
-
- public List getChildren()
- {
- return children;
- }
-
- public StreamingSubPlan withBucketToPartition(Optional bucketToPartition)
- {
- return new StreamingSubPlan(fragment.withBucketToPartition(bucketToPartition), children);
- }
+ ListMultimap stageExecutions = getStageExecutions();
+ return buildStageInfo(plan, stageExecutions);
}
- private static class StageExecutionAndScheduler
+ private StageInfo buildStageInfo(SubPlan subPlan, ListMultimap stageExecutions)
{
- private final SqlStageExecution stageExecution;
- private final StageLinkage stageLinkage;
- private final StageScheduler stageScheduler;
+ StageId stageId = getStageId(subPlan.getFragment().getId());
+ List attempts = stageExecutions.get(stageId);
+
+ StageExecutionInfo latestAttemptInfo = attempts.isEmpty() ?
+ unscheduledExecutionInfo(stageId.getId(), queryStateMachine.isDone()) :
+ getLast(attempts).getStageExecutionInfo();
+ List previousAttemptInfos = attempts.size() < 2 ?
+ ImmutableList.of() :
+ attempts.subList(0, attempts.size() - 1).stream()
+ .map(SqlStageExecution::getStageExecutionInfo)
+ .collect(toImmutableList());
- private StageExecutionAndScheduler(SqlStageExecution stageExecution, StageLinkage stageLinkage, StageScheduler stageScheduler)
- {
- this.stageExecution = requireNonNull(stageExecution, "stageExecution is null");
- this.stageLinkage = requireNonNull(stageLinkage, "stageLinkage is null");
- this.stageScheduler = requireNonNull(stageScheduler, "stageScheduler is null");
- }
+ return new StageInfo(
+ stageId,
+ locationFactory.createStageLocation(stageId),
+ Optional.of(subPlan.getFragment()),
+ latestAttemptInfo,
+ previousAttemptInfos,
+ subPlan.getChildren().stream()
+ .map(plan -> buildStageInfo(plan, stageExecutions))
+ .collect(toImmutableList()));
+ }
- public SqlStageExecution getStageExecution()
- {
- return stageExecution;
+ private ListMultimap getStageExecutions()
+ {
+ ImmutableListMultimap.Builder result = ImmutableListMultimap.builder();
+ for (Collection sectionExecutionAttempts : sectionExecutions.values()) {
+ for (SectionExecution sectionExecution : sectionExecutionAttempts) {
+ for (StageExecutionAndScheduler stageExecution : sectionExecution.getSectionStages()) {
+ result.put(stageExecution.getStageExecution().getStageExecutionId().getStageId(), stageExecution.getStageExecution());
+ }
+ }
}
+ return result.build();
+ }
- public StageLinkage getStageLinkage()
- {
- return stageLinkage;
+ @Override
+ public void cancelStage(StageId stageId)
+ {
+ try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
+ getAllStagesExecutions()
+ .filter(execution -> execution.getStageExecutionId().getStageId().equals(stageId))
+ .forEach(SqlStageExecution::cancel);
}
+ }
- public StageScheduler getStageScheduler()
- {
- return stageScheduler;
+ @Override
+ public void abort()
+ {
+ try (SetThreadName ignored = new SetThreadName("Query-%s", queryStateMachine.getQueryId())) {
+ checkState(queryStateMachine.isDone(), "query scheduler is expected to be aborted only if the query is finished: %s", queryStateMachine.getQueryState());
+ getAllStagesExecutions().forEach(SqlStageExecution::abort);
}
}
}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQuerySchedulerInterface.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQuerySchedulerInterface.java
new file mode 100644
index 0000000000000..33c9ff7497153
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQuerySchedulerInterface.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.execution.BasicStageExecutionStats;
+import com.facebook.presto.execution.StageId;
+import com.facebook.presto.execution.StageInfo;
+import io.airlift.units.Duration;
+
+public interface SqlQuerySchedulerInterface
+{
+ void start();
+
+ long getUserMemoryReservation();
+
+ long getTotalMemoryReservation();
+
+ Duration getTotalCpuTime();
+
+ BasicStageExecutionStats getBasicStageStats();
+
+ StageInfo getStageInfo();
+
+ void cancelStage(StageId stageId);
+
+ void abort();
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageExecutionAndScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageExecutionAndScheduler.java
new file mode 100644
index 0000000000000..cb4fbd013a5d4
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageExecutionAndScheduler.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.execution.SqlStageExecution;
+
+import static java.util.Objects.requireNonNull;
+
+public class StageExecutionAndScheduler
+{
+ private final SqlStageExecution stageExecution;
+ private final StageLinkage stageLinkage;
+ private final StageScheduler stageScheduler;
+
+ StageExecutionAndScheduler(SqlStageExecution stageExecution, StageLinkage stageLinkage, StageScheduler stageScheduler)
+ {
+ this.stageExecution = requireNonNull(stageExecution, "stageExecution is null");
+ this.stageLinkage = requireNonNull(stageLinkage, "stageLinkage is null");
+ this.stageScheduler = requireNonNull(stageScheduler, "stageScheduler is null");
+ }
+
+ public SqlStageExecution getStageExecution()
+ {
+ return stageExecution;
+ }
+
+ public StageLinkage getStageLinkage()
+ {
+ return stageLinkage;
+ }
+
+ public StageScheduler getStageScheduler()
+ {
+ return stageScheduler;
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageLinkage.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageLinkage.java
new file mode 100644
index 0000000000000..9b16bd4064716
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StageLinkage.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.execution.RemoteTask;
+import com.facebook.presto.execution.SqlStageExecution;
+import com.facebook.presto.execution.StageExecutionState;
+import com.facebook.presto.execution.buffer.OutputBuffers;
+import com.facebook.presto.sql.planner.PartitioningHandle;
+import com.facebook.presto.sql.planner.plan.PlanFragmentId;
+import com.google.common.primitives.Ints;
+
+import java.util.List;
+import java.util.Set;
+
+import static com.facebook.presto.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION;
+import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+
+public class StageLinkage
+{
+ private final PlanFragmentId currentStageFragmentId;
+ private final ExchangeLocationsConsumer parent;
+ private final Set childOutputBufferManagers;
+
+ public StageLinkage(PlanFragmentId fragmentId, ExchangeLocationsConsumer parent, Set children)
+ {
+ this.currentStageFragmentId = fragmentId;
+ this.parent = parent;
+ this.childOutputBufferManagers = children.stream()
+ .map(childStage -> {
+ PartitioningHandle partitioningHandle = childStage.getFragment().getPartitioningScheme().getPartitioning().getHandle();
+ if (partitioningHandle.equals(FIXED_BROADCAST_DISTRIBUTION)) {
+ return new BroadcastOutputBufferManager(childStage::setOutputBuffers);
+ }
+ else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) {
+ return new ScaledOutputBufferManager(childStage::setOutputBuffers);
+ }
+ else {
+ int partitionCount = Ints.max(childStage.getFragment().getPartitioningScheme().getBucketToPartition().get()) + 1;
+ return new PartitionedOutputBufferManager(partitioningHandle, partitionCount, childStage::setOutputBuffers);
+ }
+ })
+ .collect(toImmutableSet());
+ }
+
+ public void processScheduleResults(StageExecutionState newState, Set newTasks)
+ {
+ boolean noMoreTasks = false;
+ switch (newState) {
+ case PLANNED:
+ case SCHEDULING:
+ // workers are still being added to the query
+ break;
+ case FINISHED_TASK_SCHEDULING:
+ case SCHEDULING_SPLITS:
+ case SCHEDULED:
+ case RUNNING:
+ case FINISHED:
+ case CANCELED:
+ // no more workers will be added to the query
+ noMoreTasks = true;
+ case ABORTED:
+ case FAILED:
+ // DO NOT complete a FAILED or ABORTED stage. This will cause the
+ // stage above to finish normally, which will result in a query
+ // completing successfully when it should fail..
+ break;
+ }
+
+ // Add an exchange location to the parent stage for each new task
+ parent.addExchangeLocations(currentStageFragmentId, newTasks, noMoreTasks);
+
+ if (!childOutputBufferManagers.isEmpty()) {
+ // Add an output buffer to the child stages for each new task
+ List newOutputBuffers = newTasks.stream()
+ .map(task -> new OutputBuffers.OutputBufferId(task.getTaskId().getId()))
+ .collect(toImmutableList());
+ for (OutputBufferManager child : childOutputBufferManagers) {
+ child.addOutputBuffers(newOutputBuffers, noMoreTasks);
+ }
+ }
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingPlanSection.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingPlanSection.java
new file mode 100644
index 0000000000000..827d8ebbfa188
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingPlanSection.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.sql.planner.SubPlan;
+import com.facebook.presto.sql.planner.plan.PlanFragmentId;
+import com.facebook.presto.sql.planner.plan.RemoteSourceNode;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Set;
+
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+import static java.util.Objects.requireNonNull;
+
+public class StreamingPlanSection
+{
+ private final StreamingSubPlan plan;
+ // materialized exchange children
+ private final List children;
+
+ public StreamingPlanSection(StreamingSubPlan plan, List children)
+ {
+ this.plan = requireNonNull(plan, "plan is null");
+ this.children = ImmutableList.copyOf(requireNonNull(children, "children is null"));
+ }
+
+ public StreamingSubPlan getPlan()
+ {
+ return plan;
+ }
+
+ public List getChildren()
+ {
+ return children;
+ }
+
+ public static StreamingPlanSection extractStreamingSections(SubPlan subPlan)
+ {
+ ImmutableList.Builder materializedExchangeChildren = ImmutableList.builder();
+ StreamingSubPlan streamingSection = extractStreamingSection(subPlan, materializedExchangeChildren);
+ return new StreamingPlanSection(
+ streamingSection,
+ materializedExchangeChildren.build().stream()
+ .map(StreamingPlanSection::extractStreamingSections)
+ .collect(toImmutableList()));
+ }
+
+ private static StreamingSubPlan extractStreamingSection(SubPlan subPlan, ImmutableList.Builder materializedExchangeChildren)
+ {
+ ImmutableList.Builder streamingSources = ImmutableList.builder();
+ Set streamingFragmentIds = subPlan.getFragment().getRemoteSourceNodes().stream()
+ .map(RemoteSourceNode::getSourceFragmentIds)
+ .flatMap(List::stream)
+ .collect(toImmutableSet());
+ for (SubPlan child : subPlan.getChildren()) {
+ if (streamingFragmentIds.contains(child.getFragment().getId())) {
+ streamingSources.add(extractStreamingSection(child, materializedExchangeChildren));
+ }
+ else {
+ materializedExchangeChildren.add(child);
+ }
+ }
+ return new StreamingSubPlan(subPlan.getFragment(), streamingSources.build());
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingSubPlan.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingSubPlan.java
new file mode 100644
index 0000000000000..6edc048387da3
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/StreamingSubPlan.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.execution.scheduler;
+
+import com.facebook.presto.sql.planner.PlanFragment;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * StreamingSubPlan is similar to SubPlan but only contains streaming children
+ */
+public class StreamingSubPlan
+{
+ private final PlanFragment fragment;
+ // streaming children
+ private final List children;
+
+ public StreamingSubPlan(PlanFragment fragment, List children)
+ {
+ this.fragment = requireNonNull(fragment, "fragment is null");
+ this.children = ImmutableList.copyOf(requireNonNull(children, "children is null"));
+ }
+
+ public PlanFragment getFragment()
+ {
+ return fragment;
+ }
+
+ public List getChildren()
+ {
+ return children;
+ }
+
+ public StreamingSubPlan withBucketToPartition(Optional bucketToPartition)
+ {
+ return new StreamingSubPlan(fragment.withBucketToPartition(bucketToPartition), children);
+ }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/TableWriteInfo.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/TableWriteInfo.java
index 7b73ad8c3b302..ad348622e4b81 100644
--- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/TableWriteInfo.java
+++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/TableWriteInfo.java
@@ -15,7 +15,6 @@
package com.facebook.presto.execution.scheduler;
import com.facebook.presto.Session;
-import com.facebook.presto.execution.scheduler.SqlQueryScheduler.StreamingSubPlan;
import com.facebook.presto.metadata.AnalyzeTableHandle;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.TableLayoutResult;
diff --git a/presto-main/src/main/java/com/facebook/presto/failureDetector/FailureDetectorConfig.java b/presto-main/src/main/java/com/facebook/presto/failureDetector/FailureDetectorConfig.java
index 1e98d9f7fe2d7..4b349bdcfb25b 100644
--- a/presto-main/src/main/java/com/facebook/presto/failureDetector/FailureDetectorConfig.java
+++ b/presto-main/src/main/java/com/facebook/presto/failureDetector/FailureDetectorConfig.java
@@ -20,6 +20,7 @@
import javax.validation.constraints.DecimalMax;
import javax.validation.constraints.DecimalMin;
+import javax.validation.constraints.Min;
import javax.validation.constraints.NotNull;
import java.util.concurrent.TimeUnit;
@@ -31,6 +32,7 @@ public class FailureDetectorConfig
private Duration heartbeatInterval = new Duration(500, TimeUnit.MILLISECONDS);
private Duration warmupInterval = new Duration(5, TimeUnit.SECONDS);
private Duration expirationGraceInterval = new Duration(10, TimeUnit.MINUTES);
+ private int exponentialDecaySeconds = 60;
@NotNull
public Duration getExpirationGraceInterval()
@@ -99,4 +101,17 @@ public FailureDetectorConfig setFailureRatioThreshold(double threshold)
this.failureRatioThreshold = threshold;
return this;
}
+
+ @Min(1)
+ public int getExponentialDecaySeconds()
+ {
+ return exponentialDecaySeconds;
+ }
+
+ @Config("failure-detector.exponential-decay-seconds")
+ public FailureDetectorConfig setExponentialDecaySeconds(int exponentialDecaySeconds)
+ {
+ this.exponentialDecaySeconds = exponentialDecaySeconds;
+ return this;
+ }
}
diff --git a/presto-main/src/main/java/com/facebook/presto/failureDetector/HeartbeatFailureDetector.java b/presto-main/src/main/java/com/facebook/presto/failureDetector/HeartbeatFailureDetector.java
index 18db00175d1b0..235d881a36c0d 100644
--- a/presto-main/src/main/java/com/facebook/presto/failureDetector/HeartbeatFailureDetector.java
+++ b/presto-main/src/main/java/com/facebook/presto/failureDetector/HeartbeatFailureDetector.java
@@ -94,6 +94,7 @@ public class HeartbeatFailureDetector
private final boolean isEnabled;
private final Duration warmupInterval;
private final Duration gcGraceInterval;
+ private final int exponentialDecaySeconds;
private final boolean httpsRequired;
private final AtomicBoolean started = new AtomicBoolean();
@@ -120,6 +121,7 @@ public HeartbeatFailureDetector(
this.heartbeat = failureDetectorConfig.getHeartbeatInterval();
this.warmupInterval = failureDetectorConfig.getWarmupInterval();
this.gcGraceInterval = failureDetectorConfig.getExpirationGraceInterval();
+ this.exponentialDecaySeconds = failureDetectorConfig.getExponentialDecaySeconds();
this.isEnabled = failureDetectorConfig.isEnabled();
@@ -301,7 +303,7 @@ private MonitoringTask(ServiceDescriptor service, URI uri)
{
this.uri = uri;
this.service = service;
- this.stats = new Stats(uri);
+ this.stats = new Stats(uri, exponentialDecaySeconds);
}
public Stats getStats()
@@ -404,9 +406,9 @@ public static class Stats
private final long start = System.nanoTime();
private final URI uri;
- private final DecayCounter recentRequests = new DecayCounter(ExponentialDecay.oneMinute());
- private final DecayCounter recentFailures = new DecayCounter(ExponentialDecay.oneMinute());
- private final DecayCounter recentSuccesses = new DecayCounter(ExponentialDecay.oneMinute());
+ private final DecayCounter recentRequests;
+ private final DecayCounter recentFailures;
+ private final DecayCounter recentSuccesses;
private final AtomicReference lastRequestTime = new AtomicReference<>();
private final AtomicReference lastResponseTime = new AtomicReference<>();
private final AtomicReference lastFailureException = new AtomicReference<>();
@@ -414,9 +416,12 @@ public static class Stats
@GuardedBy("this")
private final Map, DecayCounter> failureCountByType = new HashMap<>();
- public Stats(URI uri)
+ public Stats(URI uri, int exponentialDecaySeconds)
{
this.uri = uri;
+ this.recentRequests = new DecayCounter(ExponentialDecay.seconds(exponentialDecaySeconds));
+ this.recentFailures = new DecayCounter(ExponentialDecay.seconds(exponentialDecaySeconds));
+ this.recentSuccesses = new DecayCounter(ExponentialDecay.seconds(exponentialDecaySeconds));
}
public void recordStart()
diff --git a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java
index 2784a7678fca7..142970bc88a48 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java
@@ -77,6 +77,7 @@
import com.facebook.presto.execution.scheduler.AllAtOnceExecutionPolicy;
import com.facebook.presto.execution.scheduler.ExecutionPolicy;
import com.facebook.presto.execution.scheduler.PhasedExecutionPolicy;
+import com.facebook.presto.execution.scheduler.SectionExecutionFactory;
import com.facebook.presto.execution.scheduler.SplitSchedulerStats;
import com.facebook.presto.failureDetector.FailureDetectorModule;
import com.facebook.presto.memory.ClusterMemoryManager;
@@ -294,6 +295,7 @@ protected void setup(Binder binder)
binder.bind(SplitSchedulerStats.class).in(Scopes.SINGLETON);
newExporter(binder).export(SplitSchedulerStats.class).withGeneratedName();
binder.bind(SqlQueryExecutionFactory.class).in(Scopes.SINGLETON);
+ binder.bind(SectionExecutionFactory.class).in(Scopes.SINGLETON);
getAllQueryTypes().entrySet().stream()
.filter(entry -> entry.getValue() != QueryType.DATA_DEFINITION)
.forEach(entry -> executionBinder.addBinding(entry.getKey()).to(SqlQueryExecutionFactory.class).in(Scopes.SINGLETON));
diff --git a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
index 3e14a39670fa4..edbac0afb7eba 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
@@ -232,11 +232,6 @@ public TestingPrestoServer(
.put("task.max-worker-threads", "4")
.put("exchange.client-threads", "4");
- if (coordinator) {
- // TODO: enable failure detector
- serverProperties.put("failure-detector.enabled", "false");
- }
-
ImmutableList.Builder