15
15
16
16
import com .google .common .util .concurrent .AbstractFuture ;
17
17
import com .google .common .util .concurrent .ListenableFuture ;
18
- import com .google .common .util .concurrent .ListeningScheduledExecutorService ;
19
- import io .airlift .concurrent .ThreadPoolExecutorMBean ;
20
18
import io .prestosql .Session ;
21
19
import io .prestosql .execution .QueryIdGenerator ;
22
20
import io .prestosql .execution .QueryInfo ;
39
37
import io .prestosql .transaction .TransactionManager ;
40
38
import org .weakref .jmx .Flatten ;
41
39
import org .weakref .jmx .Managed ;
42
- import org .weakref .jmx .Nested ;
43
40
44
41
import javax .inject .Inject ;
45
42
46
43
import java .util .List ;
47
44
import java .util .Optional ;
48
- import java .util .concurrent .ScheduledExecutorService ;
49
- import java .util .concurrent .ThreadPoolExecutor ;
45
+ import java .util .concurrent .Executor ;
50
46
51
47
import static com .google .common .base .Preconditions .checkArgument ;
52
48
import static com .google .common .collect .ImmutableList .toImmutableList ;
53
49
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 ;
56
50
import static io .prestosql .spi .StandardErrorCode .QUERY_TEXT_TOO_LARGE ;
57
51
import static io .prestosql .util .StatementUtils .getQueryType ;
58
52
import static io .prestosql .util .StatementUtils .isTransactionControlStatement ;
59
53
import static java .lang .String .format ;
60
54
import static java .util .Objects .requireNonNull ;
61
- import static java .util .concurrent .Executors .newScheduledThreadPool ;
62
55
63
56
public class DispatchManager
64
57
{
@@ -74,8 +67,7 @@ public class DispatchManager
74
67
75
68
private final int maxQueryLength ;
76
69
77
- private final ListeningScheduledExecutorService queryManagementExecutor ;
78
- private final ThreadPoolExecutorMBean queryManagementExecutorMBean ;
70
+ private final Executor queryExecutor ;
79
71
80
72
private final QueryTracker <DispatchQuery > queryTracker ;
81
73
@@ -92,7 +84,8 @@ public DispatchManager(
92
84
AccessControl accessControl ,
93
85
SessionSupplier sessionSupplier ,
94
86
SessionPropertyDefaults sessionPropertyDefaults ,
95
- QueryManagerConfig queryManagerConfig )
87
+ QueryManagerConfig queryManagerConfig ,
88
+ DispatchExecutor dispatchExecutor )
96
89
{
97
90
this .queryIdGenerator = requireNonNull (queryIdGenerator , "queryIdGenerator is null" );
98
91
this .queryPreparer = requireNonNull (queryPreparer , "queryPreparer is null" );
@@ -107,18 +100,9 @@ public DispatchManager(
107
100
requireNonNull (queryManagerConfig , "queryManagerConfig is null" );
108
101
this .maxQueryLength = queryManagerConfig .getMaxQueryLength ();
109
102
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 ();
113
104
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 ());
122
106
}
123
107
124
108
@ Managed
@@ -142,7 +126,7 @@ public ListenableFuture<?> createQuery(QueryId queryId, String slug, SessionCont
142
126
checkArgument (!queryTracker .tryGetQuery (queryId ).isPresent (), "query %s already exists" , queryId );
143
127
144
128
DispatchQueryCreationFuture queryCreationFuture = new DispatchQueryCreationFuture ();
145
- queryManagementExecutor . submit (() -> {
129
+ queryExecutor . execute (() -> {
146
130
try {
147
131
createQueryInternal (queryId , slug , sessionContext , query , resourceGroupManager );
148
132
}
@@ -194,13 +178,12 @@ private <C> void createQueryInternal(QueryId queryId, String slug, SessionContex
194
178
query ,
195
179
preparedQuery ,
196
180
slug ,
197
- selectionContext .getResourceGroupId (),
198
- queryManagementExecutor );
181
+ selectionContext .getResourceGroupId ());
199
182
200
183
boolean queryAdded = queryCreated (dispatchQuery );
201
184
if (queryAdded && !dispatchQuery .isDone ()) {
202
185
try {
203
- resourceGroupManager .submit (dispatchQuery , selectionContext , queryManagementExecutor );
186
+ resourceGroupManager .submit (dispatchQuery , selectionContext , queryExecutor );
204
187
}
205
188
catch (Throwable e ) {
206
189
// dispatch query has already been registered, so just fail it directly
0 commit comments