/
ControlMessageHandler.java
263 lines (231 loc) · 10.6 KB
/
ControlMessageHandler.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.drill.exec.work.batch;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.DrillBuf;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.proto.BitControl.CustomMessage;
import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
import org.apache.drill.exec.proto.BitControl.FragmentStatus;
import org.apache.drill.exec.proto.BitControl.InitializeFragments;
import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.rpc.Acks;
import org.apache.drill.exec.rpc.RequestHandler;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.ResponseSender;
import org.apache.drill.exec.rpc.RpcConstants;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.UserRpcException;
import org.apache.drill.exec.rpc.control.ControlConnection;
import org.apache.drill.exec.rpc.control.ControlRpcConfig;
import org.apache.drill.exec.rpc.control.CustomHandlerRegistry;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.work.WorkManager.WorkerBee;
import org.apache.drill.exec.work.foreman.Foreman;
import org.apache.drill.exec.work.fragment.FragmentExecutor;
import org.apache.drill.exec.work.fragment.FragmentManager;
import org.apache.drill.exec.work.fragment.FragmentStatusReporter;
import org.apache.drill.exec.work.fragment.NonRootFragmentManager;
import static org.apache.drill.exec.rpc.RpcBus.get;
public class ControlMessageHandler implements RequestHandler<ControlConnection> {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlMessageHandler.class);
private final WorkerBee bee;
private final CustomHandlerRegistry handlerRegistry = new CustomHandlerRegistry();
public ControlMessageHandler(final WorkerBee bee) {
this.bee = bee;
}
@Override
public void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
ResponseSender sender) throws RpcException {
if (RpcConstants.EXTRA_DEBUGGING) {
logger.debug("Received bit com message of type {}", rpcType);
}
switch (rpcType) {
case RpcType.REQ_CANCEL_FRAGMENT_VALUE: {
final FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
cancelFragment(handle);
sender.send(ControlRpcConfig.OK);
break;
}
case RpcType.REQ_CUSTOM_VALUE: {
final CustomMessage customMessage = get(pBody, CustomMessage.PARSER);
sender.send(handlerRegistry.handle(customMessage, (DrillBuf) dBody));
break;
}
case RpcType.REQ_RECEIVER_FINISHED_VALUE: {
final FinishedReceiver finishedReceiver = get(pBody, FinishedReceiver.PARSER);
receivingFragmentFinished(finishedReceiver);
sender.send(ControlRpcConfig.OK);
break;
}
case RpcType.REQ_FRAGMENT_STATUS_VALUE:
bee.getContext().getWorkBus().statusUpdate( get(pBody, FragmentStatus.PARSER));
// TODO: Support a type of message that has no response.
sender.send(ControlRpcConfig.OK);
break;
case RpcType.REQ_QUERY_CANCEL_VALUE: {
final QueryId queryId = get(pBody, QueryId.PARSER);
final Foreman foreman = bee.getForemanForQueryId(queryId);
if (foreman != null) {
foreman.cancel();
sender.send(ControlRpcConfig.OK);
} else {
sender.send(ControlRpcConfig.FAIL);
}
break;
}
case RpcType.REQ_INITIALIZE_FRAGMENTS_VALUE: {
final InitializeFragments fragments = get(pBody, InitializeFragments.PARSER);
final DrillbitContext drillbitContext = bee.getContext();
for(int i = 0; i < fragments.getFragmentCount(); i++) {
startNewFragment(fragments.getFragment(i), drillbitContext);
}
sender.send(ControlRpcConfig.OK);
break;
}
case RpcType.REQ_QUERY_STATUS_VALUE: {
final QueryId queryId = get(pBody, QueryId.PARSER);
final Foreman foreman = bee.getForemanForQueryId(queryId);
if (foreman == null) {
throw new RpcException("Query not running on node.");
}
final QueryProfile profile = foreman.getQueryManager().getQueryProfile();
sender.send(new Response(RpcType.RESP_QUERY_STATUS, profile));
break;
}
case RpcType.REQ_UNPAUSE_FRAGMENT_VALUE: {
final FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
resumeFragment(handle);
sender.send(ControlRpcConfig.OK);
break;
}
default:
throw new RpcException("Not yet supported.");
}
}
/**
* Start a new fragment on this node. These fragments can be leaf or intermediate fragments
* which are scheduled by remote or local Foreman node.
* @param fragment
* @throws UserRpcException
*/
private void startNewFragment(final PlanFragment fragment, final DrillbitContext drillbitContext)
throws UserRpcException {
logger.debug("Received remote fragment start instruction", fragment);
try {
final FragmentContext fragmentContext = new FragmentContext(drillbitContext, fragment,
drillbitContext.getFunctionImplementationRegistry());
final FragmentStatusReporter statusReporter = new FragmentStatusReporter(fragmentContext);
final FragmentExecutor fragmentExecutor = new FragmentExecutor(fragmentContext, fragment, statusReporter);
// we either need to start the fragment if it is a leaf fragment, or set up a fragment manager if it is non leaf.
if (fragment.getLeafFragment()) {
bee.addFragmentRunner(fragmentExecutor);
} else {
// isIntermediate, store for incoming data.
final NonRootFragmentManager manager = new NonRootFragmentManager(fragment, fragmentExecutor, statusReporter);
drillbitContext.getWorkBus().addFragmentManager(manager);
}
} catch (final ExecutionSetupException ex) {
throw new UserRpcException(drillbitContext.getEndpoint(), "Failed to create fragment context", ex);
} catch (final Exception e) {
throw new UserRpcException(drillbitContext.getEndpoint(),
"Failure while trying to start remote fragment", e);
} catch (final OutOfMemoryError t) {
if (t.getMessage().startsWith("Direct buffer")) {
throw new UserRpcException(drillbitContext.getEndpoint(),
"Out of direct memory while trying to start remote fragment", t);
} else {
throw t;
}
}
}
/* (non-Javadoc)
* @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
*/
private Ack cancelFragment(final FragmentHandle handle) {
/**
* For case 1, see {@link org.apache.drill.exec.work.foreman.QueryManager#cancelExecutingFragments}.
* In comments below, "active" refers to fragment states: SENDING, AWAITING_ALLOCATION, RUNNING and
* "inactive" refers to FINISHED, CANCELLATION_REQUESTED, CANCELLED, FAILED
*/
// Case 2: Cancel active intermediate fragment. Such a fragment will be in the work bus. Delegate cancel to the
// work bus.
final boolean removed = bee.getContext().getWorkBus().removeFragmentManager(handle, true);
if (removed) {
return Acks.OK;
}
// Case 3: Cancel active leaf fragment. Such a fragment will be with the worker bee if and only if it is running.
// Cancel directly in this case.
final FragmentExecutor runner = bee.getFragmentRunner(handle);
if (runner != null) {
runner.cancel();
return Acks.OK;
}
// Other cases: Fragment completed or does not exist. Currently known cases:
// (1) Leaf or intermediate fragment that is inactive: although we should not receive a cancellation
// request; it is possible that before the fragment state was updated in the QueryManager, this handler
// received a cancel signal.
// (2) Unknown fragment.
logger.warn("Dropping request to cancel fragment. {} does not exist.", QueryIdHelper.getQueryIdentifier(handle));
return Acks.OK;
}
private Ack resumeFragment(final FragmentHandle handle) {
// resume a pending fragment
final FragmentManager manager = bee.getContext().getWorkBus().getFragmentManager(handle);
if (manager != null) {
manager.unpause();
return Acks.OK;
}
// resume a paused fragment
final FragmentExecutor runner = bee.getFragmentRunner(handle);
if (runner != null) {
runner.unpause();
return Acks.OK;
}
// fragment completed or does not exist
logger.warn("Dropping request to resume fragment. {} does not exist.", QueryIdHelper.getQueryIdentifier(handle));
return Acks.OK;
}
private Ack receivingFragmentFinished(final FinishedReceiver finishedReceiver) {
final FragmentManager manager = bee.getContext().getWorkBus().getFragmentManager(finishedReceiver.getSender());
if (manager != null) {
manager.receivingFragmentFinished(finishedReceiver.getReceiver());
} else {
final FragmentExecutor executor = bee.getFragmentRunner(finishedReceiver.getSender());
if (executor != null) {
executor.receivingFragmentFinished(finishedReceiver.getReceiver());
} else {
logger.warn(
"Dropping request for early fragment termination for path {} -> {} as path to executor unavailable.",
QueryIdHelper.getQueryIdentifier(finishedReceiver.getSender()),
QueryIdHelper.getQueryIdentifier(finishedReceiver.getReceiver()));
}
}
return Acks.OK;
}
public CustomHandlerRegistry getHandlerRegistry() {
return handlerRegistry;
}
}