/
SqlTask.java
385 lines (333 loc) · 13.5 KB
/
SqlTask.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
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
/*
* 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;
import com.facebook.presto.OutputBuffers;
import com.facebook.presto.Session;
import com.facebook.presto.TaskSource;
import com.facebook.presto.execution.StateMachine.StateChangeListener;
import com.facebook.presto.memory.QueryContext;
import com.facebook.presto.operator.TaskContext;
import com.facebook.presto.operator.TaskStats;
import com.facebook.presto.sql.planner.PlanFragment;
import com.facebook.presto.sql.planner.plan.PlanNodeId;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import io.airlift.concurrent.SetThreadName;
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.net.URI;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import static com.facebook.presto.util.Failures.toFailures;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.CompletableFuture.completedFuture;
public class SqlTask
{
private static final Logger log = Logger.get(SqlTask.class);
private final TaskId taskId;
private final String taskInstanceId;
private final URI location;
private final TaskStateMachine taskStateMachine;
private final SharedBuffer sharedBuffer;
private final QueryContext queryContext;
private final SqlTaskExecutionFactory sqlTaskExecutionFactory;
private final AtomicReference<DateTime> lastHeartbeat = new AtomicReference<>(DateTime.now());
private final AtomicLong nextTaskInfoVersion = new AtomicLong(TaskInfo.STARTING_VERSION);
private final AtomicReference<TaskHolder> taskHolderReference = new AtomicReference<>(new TaskHolder());
private final AtomicBoolean needsPlan = new AtomicBoolean(true);
public SqlTask(
TaskId taskId,
URI location,
QueryContext queryContext,
SqlTaskExecutionFactory sqlTaskExecutionFactory,
ExecutorService taskNotificationExecutor,
final Function<SqlTask, ?> onDone,
DataSize maxBufferSize)
{
this.taskId = requireNonNull(taskId, "taskId is null");
this.taskInstanceId = UUID.randomUUID().toString();
this.location = requireNonNull(location, "location is null");
this.queryContext = requireNonNull(queryContext, "queryContext is null");
this.sqlTaskExecutionFactory = requireNonNull(sqlTaskExecutionFactory, "sqlTaskExecutionFactory is null");
requireNonNull(taskNotificationExecutor, "taskNotificationExecutor is null");
requireNonNull(onDone, "onDone is null");
requireNonNull(maxBufferSize, "maxBufferSize is null");
sharedBuffer = new SharedBuffer(taskId, taskInstanceId, taskNotificationExecutor, maxBufferSize, new UpdateSystemMemory(queryContext));
taskStateMachine = new TaskStateMachine(taskId, taskNotificationExecutor);
taskStateMachine.addStateChangeListener(new StateChangeListener<TaskState>()
{
@Override
public void stateChanged(TaskState newState)
{
if (!newState.isDone()) {
return;
}
// store final task info
while (true) {
TaskHolder taskHolder = taskHolderReference.get();
if (taskHolder.isFinished()) {
// another concurrent worker already set the final state
return;
}
if (taskHolderReference.compareAndSet(taskHolder, new TaskHolder(createTaskInfo(taskHolder), taskHolder.getIoStats()))) {
break;
}
}
// make sure buffers are cleaned up
if (newState == TaskState.FAILED || newState == TaskState.ABORTED) {
// don't close buffers for a failed query
// closed buffers signal to upstream tasks that everything finished cleanly
sharedBuffer.fail();
}
else {
sharedBuffer.destroy();
}
try {
onDone.apply(SqlTask.this);
}
catch (Exception e) {
log.warn(e, "Error running task cleanup callback %s", SqlTask.this.taskId);
}
}
});
}
private static final class UpdateSystemMemory
implements SystemMemoryUsageListener
{
private final QueryContext queryContext;
public UpdateSystemMemory(QueryContext queryContext)
{
this.queryContext = requireNonNull(queryContext, "queryContext is null");
}
@Override
public void updateSystemMemoryUsage(long deltaMemoryInBytes)
{
if (deltaMemoryInBytes > 0) {
queryContext.reserveSystemMemory(deltaMemoryInBytes);
}
else {
queryContext.freeSystemMemory(-deltaMemoryInBytes);
}
}
}
public SqlTaskIoStats getIoStats()
{
return taskHolderReference.get().getIoStats();
}
public TaskId getTaskId()
{
return taskStateMachine.getTaskId();
}
public void recordHeartbeat()
{
lastHeartbeat.set(DateTime.now());
}
public TaskInfo getTaskInfo()
{
try (SetThreadName ignored = new SetThreadName("Task-%s", taskId)) {
return createTaskInfo(taskHolderReference.get());
}
}
private TaskInfo createTaskInfo(TaskHolder taskHolder)
{
// Always return a new TaskInfo with a larger version number;
// otherwise a client will not accept the update
long versionNumber = nextTaskInfoVersion.getAndIncrement();
TaskState state = taskStateMachine.getState();
List<ExecutionFailureInfo> failures = ImmutableList.of();
if (state == TaskState.FAILED) {
failures = toFailures(taskStateMachine.getFailureCauses());
}
TaskStats taskStats;
Set<PlanNodeId> noMoreSplits;
TaskInfo finalTaskInfo = taskHolder.getFinalTaskInfo();
if (finalTaskInfo != null) {
taskStats = finalTaskInfo.getStats();
noMoreSplits = finalTaskInfo.getNoMoreSplits();
}
else {
SqlTaskExecution taskExecution = taskHolder.getTaskExecution();
if (taskExecution != null) {
taskStats = taskExecution.getTaskContext().getTaskStats();
noMoreSplits = taskExecution.getNoMoreSplits();
}
else {
// if the task completed without creation, set end time
DateTime endTime = state.isDone() ? DateTime.now() : null;
taskStats = new TaskStats(taskStateMachine.getCreatedTime(), endTime);
noMoreSplits = ImmutableSet.of();
}
}
return new TaskInfo(
taskStateMachine.getTaskId(),
taskInstanceId,
versionNumber,
state,
location,
lastHeartbeat.get(),
sharedBuffer.getInfo(),
noMoreSplits,
taskStats,
failures,
needsPlan.get());
}
public CompletableFuture<TaskInfo> getTaskInfo(TaskState callersCurrentState)
{
requireNonNull(callersCurrentState, "callersCurrentState is null");
// If the caller's current state is already done, just return the current
// state of this task as it will either be done or possibly still running
// (due to a bug in the caller), since we can not transition from a done
// state.
if (callersCurrentState.isDone()) {
return completedFuture(getTaskInfo());
}
CompletableFuture<TaskState> futureTaskState = taskStateMachine.getStateChange(callersCurrentState);
return futureTaskState.thenApply(input -> getTaskInfo());
}
public TaskInfo updateTask(Session session, Optional<PlanFragment> fragment, List<TaskSource> sources, OutputBuffers outputBuffers)
{
try {
// assure the task execution is only created once
SqlTaskExecution taskExecution;
synchronized (this) {
// is task already complete?
TaskHolder taskHolder = taskHolderReference.get();
if (taskHolder.isFinished()) {
return taskHolder.getFinalTaskInfo();
}
taskExecution = taskHolder.getTaskExecution();
if (taskExecution == null) {
checkState(fragment.isPresent(), "fragment must be present");
taskExecution = sqlTaskExecutionFactory.create(session, queryContext, taskStateMachine, sharedBuffer, fragment.get(), sources);
taskHolderReference.compareAndSet(taskHolder, new TaskHolder(taskExecution));
needsPlan.set(false);
}
}
if (taskExecution != null) {
// addSources checks for task completion, so update the buffers first and the task might complete earlier
sharedBuffer.setOutputBuffers(outputBuffers);
taskExecution.addSources(sources);
}
}
catch (Error e) {
failed(e);
throw e;
}
catch (RuntimeException e) {
failed(e);
}
return getTaskInfo();
}
public CompletableFuture<BufferResult> getTaskResults(TaskId outputName, long startingSequenceId, DataSize maxSize)
{
requireNonNull(outputName, "outputName is null");
checkArgument(maxSize.toBytes() > 0, "maxSize must be at least 1 byte");
return sharedBuffer.get(outputName, startingSequenceId, maxSize);
}
public TaskInfo abortTaskResults(TaskId outputId)
{
requireNonNull(outputId, "outputId is null");
log.debug("Aborting task %s output %s", taskId, outputId);
sharedBuffer.abort(outputId);
return getTaskInfo();
}
public void failed(Throwable cause)
{
requireNonNull(cause, "cause is null");
taskStateMachine.failed(cause);
}
public TaskInfo cancel()
{
taskStateMachine.cancel();
return getTaskInfo();
}
public TaskInfo abort()
{
taskStateMachine.abort();
return getTaskInfo();
}
@Override
public String toString()
{
return taskId.toString();
}
private static final class TaskHolder
{
private final SqlTaskExecution taskExecution;
private final TaskInfo finalTaskInfo;
private final SqlTaskIoStats finalIoStats;
private TaskHolder()
{
this.taskExecution = null;
this.finalTaskInfo = null;
this.finalIoStats = null;
}
private TaskHolder(SqlTaskExecution taskExecution)
{
this.taskExecution = requireNonNull(taskExecution, "taskExecution is null");
this.finalTaskInfo = null;
this.finalIoStats = null;
}
private TaskHolder(TaskInfo finalTaskInfo, SqlTaskIoStats finalIoStats)
{
this.taskExecution = null;
this.finalTaskInfo = requireNonNull(finalTaskInfo, "finalTaskInfo is null");
this.finalIoStats = requireNonNull(finalIoStats, "finalIoStats is null");
}
public boolean isFinished()
{
return finalTaskInfo != null;
}
@Nullable
public SqlTaskExecution getTaskExecution()
{
return taskExecution;
}
@Nullable
public TaskInfo getFinalTaskInfo()
{
return finalTaskInfo;
}
public SqlTaskIoStats getIoStats()
{
// if we are finished, return the final IoStats
if (finalIoStats != null) {
return finalIoStats;
}
// if we haven't started yet, return an empty IoStats
if (taskExecution == null) {
return new SqlTaskIoStats();
}
// get IoStats from the current task execution
TaskContext taskContext = taskExecution.getTaskContext();
return new SqlTaskIoStats(taskContext.getInputDataSize(), taskContext.getInputPositions(), taskContext.getOutputDataSize(), taskContext.getOutputPositions());
}
}
public void addStateChangeListener(StateChangeListener<TaskState> stateChangeListener)
{
taskStateMachine.addStateChangeListener(stateChangeListener);
}
}