Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 5463bb5

Browse files
committed
Cleanup dispatcher executor management
1 parent 0f6e81b commit 5463bb5

8 files changed

+151
-60
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,112 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package io.prestosql.dispatcher;
15+
16+
import com.google.common.io.Closer;
17+
import com.google.common.util.concurrent.ListeningExecutorService;
18+
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
19+
import io.airlift.concurrent.ThreadPoolExecutorMBean;
20+
import io.prestosql.execution.QueryManagerConfig;
21+
import org.weakref.jmx.Flatten;
22+
import org.weakref.jmx.Managed;
23+
import org.weakref.jmx.Nested;
24+
25+
import javax.annotation.PreDestroy;
26+
import javax.inject.Inject;
27+
28+
import java.util.concurrent.ExecutorService;
29+
import java.util.concurrent.ScheduledExecutorService;
30+
import java.util.concurrent.ThreadPoolExecutor;
31+
32+
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
33+
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
34+
import static java.util.Objects.requireNonNull;
35+
import static java.util.concurrent.Executors.newCachedThreadPool;
36+
import static java.util.concurrent.Executors.newScheduledThreadPool;
37+
38+
public class DispatchExecutor
39+
{
40+
private final Closer closer = Closer.create();
41+
42+
private final ListeningExecutorService executor;
43+
private final ListeningScheduledExecutorService scheduledExecutor;
44+
45+
private final DispatchExecutorMBeans mbeans;
46+
47+
@Inject
48+
public DispatchExecutor(QueryManagerConfig config)
49+
{
50+
ExecutorService coreExecutor = newCachedThreadPool(daemonThreadsNamed("dispatcher-query-%s"));
51+
closer.register(coreExecutor::shutdownNow);
52+
executor = listeningDecorator(coreExecutor);
53+
54+
ScheduledExecutorService coreScheduledExecutor = newScheduledThreadPool(config.getQueryManagerExecutorPoolSize(), daemonThreadsNamed("dispatch-executor-%s"));
55+
closer.register(coreScheduledExecutor::shutdownNow);
56+
scheduledExecutor = listeningDecorator(coreScheduledExecutor);
57+
58+
mbeans = new DispatchExecutorMBeans(coreExecutor, coreScheduledExecutor);
59+
}
60+
61+
public ListeningExecutorService getExecutor()
62+
{
63+
return executor;
64+
}
65+
66+
public ListeningScheduledExecutorService getScheduledExecutor()
67+
{
68+
return scheduledExecutor;
69+
}
70+
71+
@Managed
72+
@Flatten
73+
public DispatchExecutorMBeans getMbeans()
74+
{
75+
return mbeans;
76+
}
77+
78+
@PreDestroy
79+
public void shutdown()
80+
throws Exception
81+
{
82+
closer.close();
83+
}
84+
85+
public class DispatchExecutorMBeans
86+
{
87+
private final ThreadPoolExecutorMBean executor;
88+
private final ThreadPoolExecutorMBean scheduledExecutor;
89+
90+
public DispatchExecutorMBeans(ExecutorService coreExecutor, ScheduledExecutorService coreScheduledExecutor)
91+
{
92+
requireNonNull(coreExecutor, "coreExecutor is null");
93+
requireNonNull(coreScheduledExecutor, "coreScheduledExecutor is null");
94+
executor = new ThreadPoolExecutorMBean((ThreadPoolExecutor) coreExecutor);
95+
scheduledExecutor = new ThreadPoolExecutorMBean((ThreadPoolExecutor) coreScheduledExecutor);
96+
}
97+
98+
@Managed
99+
@Nested
100+
public ThreadPoolExecutorMBean getExecutor()
101+
{
102+
return executor;
103+
}
104+
105+
@Managed
106+
@Nested
107+
public ThreadPoolExecutorMBean getScheduledExecutor()
108+
{
109+
return scheduledExecutor;
110+
}
111+
}
112+
}

presto-main/src/main/java/io/prestosql/dispatcher/DispatchManager.java

+9-26
Original file line numberDiff line numberDiff line change
@@ -15,8 +15,6 @@
1515

1616
import com.google.common.util.concurrent.AbstractFuture;
1717
import com.google.common.util.concurrent.ListenableFuture;
18-
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
19-
import io.airlift.concurrent.ThreadPoolExecutorMBean;
2018
import io.prestosql.Session;
2119
import io.prestosql.execution.QueryIdGenerator;
2220
import io.prestosql.execution.QueryInfo;
@@ -39,26 +37,21 @@
3937
import io.prestosql.transaction.TransactionManager;
4038
import org.weakref.jmx.Flatten;
4139
import org.weakref.jmx.Managed;
42-
import org.weakref.jmx.Nested;
4340

4441
import javax.inject.Inject;
4542

4643
import java.util.List;
4744
import java.util.Optional;
48-
import java.util.concurrent.ScheduledExecutorService;
49-
import java.util.concurrent.ThreadPoolExecutor;
45+
import java.util.concurrent.Executor;
5046

5147
import static com.google.common.base.Preconditions.checkArgument;
5248
import static com.google.common.collect.ImmutableList.toImmutableList;
5349
import static com.google.common.util.concurrent.Futures.immediateFuture;
54-
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
55-
import static io.airlift.concurrent.Threads.threadsNamed;
5650
import static io.prestosql.spi.StandardErrorCode.QUERY_TEXT_TOO_LARGE;
5751
import static io.prestosql.util.StatementUtils.getQueryType;
5852
import static io.prestosql.util.StatementUtils.isTransactionControlStatement;
5953
import static java.lang.String.format;
6054
import static java.util.Objects.requireNonNull;
61-
import static java.util.concurrent.Executors.newScheduledThreadPool;
6255

6356
public class DispatchManager
6457
{
@@ -74,8 +67,7 @@ public class DispatchManager
7467

7568
private final int maxQueryLength;
7669

77-
private final ListeningScheduledExecutorService queryManagementExecutor;
78-
private final ThreadPoolExecutorMBean queryManagementExecutorMBean;
70+
private final Executor queryExecutor;
7971

8072
private final QueryTracker<DispatchQuery> queryTracker;
8173

@@ -92,7 +84,8 @@ public DispatchManager(
9284
AccessControl accessControl,
9385
SessionSupplier sessionSupplier,
9486
SessionPropertyDefaults sessionPropertyDefaults,
95-
QueryManagerConfig queryManagerConfig)
87+
QueryManagerConfig queryManagerConfig,
88+
DispatchExecutor dispatchExecutor)
9689
{
9790
this.queryIdGenerator = requireNonNull(queryIdGenerator, "queryIdGenerator is null");
9891
this.queryPreparer = requireNonNull(queryPreparer, "queryPreparer is null");
@@ -107,18 +100,9 @@ public DispatchManager(
107100
requireNonNull(queryManagerConfig, "queryManagerConfig is null");
108101
this.maxQueryLength = queryManagerConfig.getMaxQueryLength();
109102

110-
ScheduledExecutorService scheduledExecutorService = newScheduledThreadPool(queryManagerConfig.getQueryManagerExecutorPoolSize(), threadsNamed("query-dispatch-%s"));
111-
queryManagementExecutor = listeningDecorator(scheduledExecutorService);
112-
queryManagementExecutorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) scheduledExecutorService);
103+
this.queryExecutor = requireNonNull(dispatchExecutor, "dispatchExecutor is null").getExecutor();
113104

114-
this.queryTracker = new QueryTracker<>(queryManagerConfig, queryManagementExecutor);
115-
}
116-
117-
@Managed(description = "Query dispatch executor")
118-
@Nested
119-
public ThreadPoolExecutorMBean getExecutor()
120-
{
121-
return queryManagementExecutorMBean;
105+
this.queryTracker = new QueryTracker<>(queryManagerConfig, dispatchExecutor.getScheduledExecutor());
122106
}
123107

124108
@Managed
@@ -142,7 +126,7 @@ public ListenableFuture<?> createQuery(QueryId queryId, String slug, SessionCont
142126
checkArgument(!queryTracker.tryGetQuery(queryId).isPresent(), "query %s already exists", queryId);
143127

144128
DispatchQueryCreationFuture queryCreationFuture = new DispatchQueryCreationFuture();
145-
queryManagementExecutor.submit(() -> {
129+
queryExecutor.execute(() -> {
146130
try {
147131
createQueryInternal(queryId, slug, sessionContext, query, resourceGroupManager);
148132
}
@@ -194,13 +178,12 @@ private <C> void createQueryInternal(QueryId queryId, String slug, SessionContex
194178
query,
195179
preparedQuery,
196180
slug,
197-
selectionContext.getResourceGroupId(),
198-
queryManagementExecutor);
181+
selectionContext.getResourceGroupId());
199182

200183
boolean queryAdded = queryCreated(dispatchQuery);
201184
if (queryAdded && !dispatchQuery.isDone()) {
202185
try {
203-
resourceGroupManager.submit(dispatchQuery, selectionContext, queryManagementExecutor);
186+
resourceGroupManager.submit(dispatchQuery, selectionContext, queryExecutor);
204187
}
205188
catch (Throwable e) {
206189
// dispatch query has already been registered, so just fail it directly

presto-main/src/main/java/io/prestosql/dispatcher/DispatchQueryFactory.java

+1-4
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,12 @@
1717
import io.prestosql.execution.QueryPreparer.PreparedQuery;
1818
import io.prestosql.spi.resourcegroups.ResourceGroupId;
1919

20-
import java.util.concurrent.ExecutorService;
21-
2220
public interface DispatchQueryFactory
2321
{
2422
DispatchQuery createDispatchQuery(
2523
Session session,
2624
String query,
2725
PreparedQuery preparedQuery,
2826
String slug,
29-
ResourceGroupId resourceGroup,
30-
ExecutorService queryExecutor);
27+
ResourceGroupId resourceGroup);
3128
}

presto-main/src/main/java/io/prestosql/dispatcher/FailedDispatchQueryFactory.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616
import io.prestosql.Session;
1717
import io.prestosql.event.QueryMonitor;
1818
import io.prestosql.execution.ExecutionFailureInfo;
19-
import io.prestosql.execution.ForQueryExecution;
2019
import io.prestosql.execution.LocationFactory;
2120
import io.prestosql.server.BasicQueryInfo;
2221
import io.prestosql.spi.resourcegroups.ResourceGroupId;
@@ -36,11 +35,11 @@ public class FailedDispatchQueryFactory
3635
private final ExecutorService executor;
3736

3837
@Inject
39-
public FailedDispatchQueryFactory(QueryMonitor queryMonitor, LocationFactory locationFactory, @ForQueryExecution ExecutorService executor)
38+
public FailedDispatchQueryFactory(QueryMonitor queryMonitor, LocationFactory locationFactory, DispatchExecutor dispatchExecutor)
4039
{
4140
this.queryMonitor = requireNonNull(queryMonitor, "queryMonitor is null");
4241
this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
43-
this.executor = requireNonNull(executor, "executor is null");
42+
this.executor = requireNonNull(dispatchExecutor, "dispatchExecutor is null").getExecutor();
4443
}
4544

4645
public FailedDispatchQuery createFailedDispatchQuery(Session session, String query, Optional<ResourceGroupId> resourceGroup, Throwable throwable)

presto-main/src/main/java/io/prestosql/dispatcher/LocalDispatchQuery.java

+5-5
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@
3131
import org.joda.time.DateTime;
3232

3333
import java.util.Optional;
34-
import java.util.concurrent.ExecutorService;
34+
import java.util.concurrent.Executor;
3535
import java.util.function.Function;
3636

3737
import static com.google.common.util.concurrent.Futures.immediateFuture;
@@ -55,15 +55,15 @@ public class LocalDispatchQuery
5555

5656
private final ClusterSizeMonitor clusterSizeMonitor;
5757

58-
private final ExecutorService queryExecutor;
58+
private final Executor queryExecutor;
5959

6060
private final Function<QueryExecution, ListenableFuture<?>> querySubmitter;
6161

6262
public LocalDispatchQuery(
6363
QueryStateMachine stateMachine,
6464
ListenableFuture<QueryExecution> queryExecutionFuture,
6565
ClusterSizeMonitor clusterSizeMonitor,
66-
ExecutorService queryExecutor,
66+
Executor queryExecutor,
6767
Function<QueryExecution, ListenableFuture<?>> querySubmitter)
6868
{
6969
this.stateMachine = requireNonNull(stateMachine, "stateMachine is null");
@@ -88,12 +88,12 @@ private void waitForMinimumWorkers()
8888
ListenableFuture<?> minimumWorkerFuture = clusterSizeMonitor.waitForMinimumWorkers();
8989
// when worker requirement is met, wait for query execution to finish construction and then start the execution
9090
addSuccessCallback(minimumWorkerFuture, () -> addSuccessCallback(queryExecutionFuture, this::startExecution));
91-
addExceptionCallback(minimumWorkerFuture, throwable -> queryExecutor.submit(() -> stateMachine.transitionToFailed(throwable)));
91+
addExceptionCallback(minimumWorkerFuture, throwable -> queryExecutor.execute(() -> stateMachine.transitionToFailed(throwable)));
9292
}
9393

9494
private void startExecution(QueryExecution queryExecution)
9595
{
96-
queryExecutor.submit(() -> {
96+
queryExecutor.execute(() -> {
9797
if (stateMachine.transitionToDispatching()) {
9898
querySubmitter.apply(queryExecution);
9999
}

presto-main/src/main/java/io/prestosql/dispatcher/LocalDispatchQueryFactory.java

+8-13
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
import io.prestosql.Session;
1919
import io.prestosql.event.QueryMonitor;
2020
import io.prestosql.execution.ClusterSizeMonitor;
21-
import io.prestosql.execution.ForQueryExecution;
2221
import io.prestosql.execution.LocationFactory;
2322
import io.prestosql.execution.QueryExecution;
2423
import io.prestosql.execution.QueryExecution.QueryExecutionFactory;
@@ -38,9 +37,7 @@
3837

3938
import java.util.Map;
4039
import java.util.Optional;
41-
import java.util.concurrent.ExecutorService;
4240

43-
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
4441
import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED;
4542
import static io.prestosql.util.StatementUtils.isTransactionControlStatement;
4643
import static java.util.Objects.requireNonNull;
@@ -59,7 +56,7 @@ public class LocalDispatchQueryFactory
5956

6057
private final Map<Class<? extends Statement>, QueryExecutionFactory<?>> executionFactories;
6158
private final WarningCollectorFactory warningCollectorFactory;
62-
private final ListeningExecutorService executorService;
59+
private final ListeningExecutorService executor;
6360

6461
@Inject
6562
public LocalDispatchQueryFactory(
@@ -72,8 +69,7 @@ public LocalDispatchQueryFactory(
7269
Map<Class<? extends Statement>, QueryExecutionFactory<?>> executionFactories,
7370
WarningCollectorFactory warningCollectorFactory,
7471
ClusterSizeMonitor clusterSizeMonitor,
75-
@ForQueryExecution ExecutorService executorService)
76-
72+
DispatchExecutor dispatchExecutor)
7773
{
7874
this.queryManager = requireNonNull(queryManager, "queryManager is null");
7975
this.transactionManager = requireNonNull(transactionManager, "transactionManager is null");
@@ -86,7 +82,7 @@ public LocalDispatchQueryFactory(
8682

8783
this.clusterSizeMonitor = requireNonNull(clusterSizeMonitor, "clusterSizeMonitor is null");
8884

89-
this.executorService = listeningDecorator(requireNonNull(executorService, "executorService is null"));
85+
this.executor = requireNonNull(dispatchExecutor, "executorService is null").getExecutor();
9086
}
9187

9288
@Override
@@ -95,8 +91,7 @@ public DispatchQuery createDispatchQuery(
9591
String query,
9692
PreparedQuery preparedQuery,
9793
String slug,
98-
ResourceGroupId resourceGroup,
99-
ExecutorService queryExecutor)
94+
ResourceGroupId resourceGroup)
10095
{
10196
WarningCollector warningCollector = warningCollectorFactory.create();
10297
QueryStateMachine stateMachine = QueryStateMachine.begin(
@@ -107,13 +102,13 @@ public DispatchQuery createDispatchQuery(
107102
isTransactionControlStatement(preparedQuery.getStatement()),
108103
transactionManager,
109104
accessControl,
110-
executorService,
105+
executor,
111106
metadata,
112107
warningCollector);
113108

114109
queryMonitor.queryCreatedEvent(stateMachine.getBasicQueryInfo(Optional.empty()));
115110

116-
ListenableFuture<QueryExecution> queryExecutionFuture = executorService.submit(() -> {
111+
ListenableFuture<QueryExecution> queryExecutionFuture = executor.submit(() -> {
117112
QueryExecutionFactory<?> queryExecutionFactory = executionFactories.get(preparedQuery.getStatement().getClass());
118113
if (queryExecutionFactory == null) {
119114
throw new PrestoException(NOT_SUPPORTED, "Unsupported statement type: " + preparedQuery.getStatement().getClass().getSimpleName());
@@ -126,7 +121,7 @@ public DispatchQuery createDispatchQuery(
126121
stateMachine,
127122
queryExecutionFuture,
128123
clusterSizeMonitor,
129-
queryExecutor,
130-
queryExecution -> executorService.submit(() -> queryManager.createQuery(queryExecution)));
124+
executor,
125+
queryExecution -> executor.submit(() -> queryManager.createQuery(queryExecution)));
131126
}
132127
}

0 commit comments

Comments
 (0)