/
DefaultTaskExecutor.java
177 lines (152 loc) · 6.31 KB
/
DefaultTaskExecutor.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
/*
* 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.kafka.streams.processor.internals.tasks;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.internals.ReadOnlyTask;
import org.apache.kafka.streams.processor.internals.StreamTask;
import org.slf4j.Logger;
import java.time.Duration;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
public class DefaultTaskExecutor implements TaskExecutor {
private class TaskExecutorThread extends Thread {
private final AtomicBoolean isRunning = new AtomicBoolean(true);
private final AtomicReference<KafkaFutureImpl<StreamTask>> pauseRequested = new AtomicReference<>(null);
private final Logger log;
public TaskExecutorThread(final String name) {
super(name);
final String logPrefix = String.format("%s ", name);
final LogContext logContext = new LogContext(logPrefix);
log = logContext.logger(DefaultTaskExecutor.class);
}
@Override
public void run() {
log.info("Task executor thread started");
try {
while (isRunning.get()) {
runOnce(time.milliseconds());
}
} catch (final StreamsException e) {
handleException(e);
} catch (final Exception e) {
handleException(new StreamsException(e));
} finally {
if (currentTask != null) {
unassignCurrentTask();
}
shutdownGate.countDown();
log.info("Task executor thread shutdown");
}
}
private void handleException(final StreamsException e) {
if (currentTask != null) {
taskManager.setUncaughtException(e, currentTask.id());
} else {
// If we do not currently have a task assigned and still get an error, this is fatal for the executor thread
throw e;
}
}
private void runOnce(final long nowMs) {
final KafkaFutureImpl<StreamTask> pauseFuture;
if ((pauseFuture = pauseRequested.getAndSet(null)) != null) {
final StreamTask unassignedTask = unassignCurrentTask();
pauseFuture.complete(unassignedTask);
}
if (currentTask == null) {
currentTask = taskManager.assignNextTask(DefaultTaskExecutor.this);
} else {
// if a task is no longer processable, ask task-manager to give it another
// task in the next iteration
if (currentTask.isProcessable(nowMs)) {
currentTask.process(nowMs);
} else {
unassignCurrentTask();
}
}
}
private StreamTask unassignCurrentTask() {
if (currentTask == null)
throw new IllegalStateException("Does not own any task while being ask to unassign from task manager");
// flush the task before giving it back to task manager
// TODO: we can add a separate function in StreamTask to just flush and not return offsets
currentTask.prepareCommit();
taskManager.unassignTask(currentTask, DefaultTaskExecutor.this);
final StreamTask retTask = currentTask;
currentTask = null;
return retTask;
}
}
private final Time time;
private final String name;
private final TaskManager taskManager;
private StreamTask currentTask = null;
private TaskExecutorThread taskExecutorThread = null;
private CountDownLatch shutdownGate;
public DefaultTaskExecutor(final TaskManager taskManager,
final String name,
final Time time) {
this.time = time;
this.name = name;
this.taskManager = taskManager;
}
@Override
public String name() {
return name;
}
@Override
public void start() {
if (taskExecutorThread == null) {
taskExecutorThread = new TaskExecutorThread(name);
taskExecutorThread.start();
shutdownGate = new CountDownLatch(1);
}
}
@Override
public void shutdown(final Duration timeout) {
if (taskExecutorThread != null) {
taskExecutorThread.isRunning.set(false);
taskExecutorThread.interrupt();
try {
if (!shutdownGate.await(timeout.toMillis(), TimeUnit.MILLISECONDS)) {
throw new StreamsException("State updater thread did not shutdown within the timeout");
}
taskExecutorThread = null;
} catch (final InterruptedException ignored) {
}
}
}
@Override
public ReadOnlyTask currentTask() {
return currentTask != null ? new ReadOnlyTask(currentTask) : null;
}
@Override
public KafkaFuture<StreamTask> unassign() {
final KafkaFutureImpl<StreamTask> future = new KafkaFutureImpl<>();
if (taskExecutorThread != null) {
taskExecutorThread.pauseRequested.set(future);
} else {
future.complete(null);
}
return future;
}
}