/
ActiveTaskCreator.java
344 lines (311 loc) · 13.9 KB
/
ActiveTaskCreator.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
/*
* 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;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.KafkaClientSupplier;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics;
import org.apache.kafka.streams.state.internals.ThreadCache;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_ALPHA;
import static org.apache.kafka.streams.internals.StreamsConfigUtils.eosEnabled;
import static org.apache.kafka.streams.internals.StreamsConfigUtils.processingMode;
import static org.apache.kafka.streams.processor.internals.ClientUtils.getTaskProducerClientId;
import static org.apache.kafka.streams.processor.internals.ClientUtils.getThreadProducerClientId;
class ActiveTaskCreator {
private final TopologyMetadata topologyMetadata;
private final StreamsConfig applicationConfig;
private final StreamsMetricsImpl streamsMetrics;
private final StateDirectory stateDirectory;
private final ChangelogReader storeChangelogReader;
private final ThreadCache cache;
private final Time time;
private final KafkaClientSupplier clientSupplier;
private final String threadId;
private final Logger log;
private final Sensor createTaskSensor;
private final StreamsProducer threadProducer;
private final Map<TaskId, StreamsProducer> taskProducers;
private final ProcessingMode processingMode;
private final boolean stateUpdaterEnabled;
private final boolean processingThreadsEnabled;
ActiveTaskCreator(final TopologyMetadata topologyMetadata,
final StreamsConfig applicationConfig,
final StreamsMetricsImpl streamsMetrics,
final StateDirectory stateDirectory,
final ChangelogReader storeChangelogReader,
final ThreadCache cache,
final Time time,
final KafkaClientSupplier clientSupplier,
final String threadId,
final UUID processId,
final Logger log,
final boolean stateUpdaterEnabled,
final boolean processingThreadsEnabled
) {
this.topologyMetadata = topologyMetadata;
this.applicationConfig = applicationConfig;
this.streamsMetrics = streamsMetrics;
this.stateDirectory = stateDirectory;
this.storeChangelogReader = storeChangelogReader;
this.cache = cache;
this.time = time;
this.clientSupplier = clientSupplier;
this.threadId = threadId;
this.log = log;
this.stateUpdaterEnabled = stateUpdaterEnabled;
this.processingThreadsEnabled = processingThreadsEnabled;
createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics);
processingMode = processingMode(applicationConfig);
if (processingMode == EXACTLY_ONCE_ALPHA) {
threadProducer = null;
taskProducers = new HashMap<>();
} else { // non-eos and eos-v2
log.info("Creating thread producer client");
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final LogContext logContext = new LogContext(threadIdPrefix);
threadProducer = new StreamsProducer(
applicationConfig,
threadId,
clientSupplier,
null,
processId,
logContext,
time);
taskProducers = Collections.emptyMap();
}
}
public void reInitializeThreadProducer() {
threadProducer.resetProducer();
}
StreamsProducer streamsProducerForTask(final TaskId taskId) {
if (processingMode != EXACTLY_ONCE_ALPHA) {
throw new IllegalStateException("Expected EXACTLY_ONCE to be enabled, but the processing mode was " + processingMode);
}
final StreamsProducer taskProducer = taskProducers.get(taskId);
if (taskProducer == null) {
throw new IllegalStateException("Unknown TaskId: " + taskId);
}
return taskProducer;
}
StreamsProducer threadProducer() {
if (processingMode == EXACTLY_ONCE_ALPHA) {
throw new IllegalStateException("Expected AT_LEAST_ONCE or EXACTLY_ONCE_V2 to be enabled, but the processing mode was " + processingMode);
}
return threadProducer;
}
// TODO: convert to StreamTask when we remove TaskManager#StateMachineTask with mocks
public Collection<Task> createTasks(final Consumer<byte[], byte[]> consumer,
final Map<TaskId, Set<TopicPartition>> tasksToBeCreated) {
final List<Task> createdTasks = new ArrayList<>();
for (final Map.Entry<TaskId, Set<TopicPartition>> newTaskAndPartitions : tasksToBeCreated.entrySet()) {
final TaskId taskId = newTaskAndPartitions.getKey();
final LogContext logContext = getLogContext(taskId);
final Set<TopicPartition> partitions = newTaskAndPartitions.getValue();
final ProcessorTopology topology = topologyMetadata.buildSubtopology(taskId);
final ProcessorStateManager stateManager = new ProcessorStateManager(
taskId,
Task.TaskType.ACTIVE,
eosEnabled(applicationConfig),
logContext,
stateDirectory,
storeChangelogReader,
topology.storeToChangelogTopic(),
partitions,
stateUpdaterEnabled);
final InternalProcessorContext<Object, Object> context = new ProcessorContextImpl(
taskId,
applicationConfig,
stateManager,
streamsMetrics,
cache
);
createdTasks.add(
createActiveTask(
taskId,
partitions,
consumer,
logContext,
topology,
stateManager,
context
)
);
}
return createdTasks;
}
private RecordCollector createRecordCollector(final TaskId taskId,
final LogContext logContext,
final ProcessorTopology topology) {
final StreamsProducer streamsProducer;
if (processingMode == ProcessingMode.EXACTLY_ONCE_ALPHA) {
log.info("Creating producer client for task {}", taskId);
streamsProducer = new StreamsProducer(
applicationConfig,
threadId,
clientSupplier,
taskId,
null,
logContext,
time
);
taskProducers.put(taskId, streamsProducer);
} else {
streamsProducer = threadProducer;
}
return new RecordCollectorImpl(
logContext,
taskId,
streamsProducer,
applicationConfig.defaultProductionExceptionHandler(),
streamsMetrics,
topology
);
}
/*
* TODO: we pass in the new input partitions to validate if they still match,
* in the future we when we have fixed partitions -> tasks mapping,
* we should always reuse the input partition and hence no need validations
*/
StreamTask createActiveTaskFromStandby(final StandbyTask standbyTask,
final Set<TopicPartition> inputPartitions,
final Consumer<byte[], byte[]> consumer) {
if (!inputPartitions.equals(standbyTask.inputPartitions)) {
log.warn("Detected unmatched input partitions for task {} when recycling it from standby to active", standbyTask.id);
}
standbyTask.prepareRecycle();
standbyTask.stateMgr.transitionTaskType(Task.TaskType.ACTIVE, getLogContext(standbyTask.id));
final RecordCollector recordCollector = createRecordCollector(standbyTask.id, getLogContext(standbyTask.id), standbyTask.topology);
final StreamTask task = new StreamTask(
standbyTask.id,
inputPartitions,
standbyTask.topology,
consumer,
standbyTask.config,
streamsMetrics,
stateDirectory,
cache,
time,
standbyTask.stateMgr,
recordCollector,
standbyTask.processorContext,
standbyTask.logContext,
processingThreadsEnabled
);
log.trace("Created active task {} from recycled standby task with assigned partitions {}", task.id, inputPartitions);
createTaskSensor.record();
return task;
}
private StreamTask createActiveTask(final TaskId taskId,
final Set<TopicPartition> inputPartitions,
final Consumer<byte[], byte[]> consumer,
final LogContext logContext,
final ProcessorTopology topology,
final ProcessorStateManager stateManager,
final InternalProcessorContext<Object, Object> context) {
final RecordCollector recordCollector = createRecordCollector(taskId, logContext, topology);
final StreamTask task = new StreamTask(
taskId,
inputPartitions,
topology,
consumer,
topologyMetadata.getTaskConfigFor(taskId),
streamsMetrics,
stateDirectory,
cache,
time,
stateManager,
recordCollector,
context,
logContext,
processingThreadsEnabled
);
log.trace("Created active task {} with assigned partitions {}", taskId, inputPartitions);
createTaskSensor.record();
return task;
}
void closeThreadProducerIfNeeded() {
if (threadProducer != null) {
try {
threadProducer.close();
} catch (final RuntimeException e) {
throw new StreamsException("Thread producer encounter error trying to close.", e);
}
}
}
void closeAndRemoveTaskProducerIfNeeded(final TaskId id) {
final StreamsProducer taskProducer = taskProducers.remove(id);
if (taskProducer != null) {
try {
taskProducer.close();
} catch (final RuntimeException e) {
throw new StreamsException("[" + id + "] task producer encounter error trying to close.", e, id);
}
}
}
Map<MetricName, Metric> producerMetrics() {
// When EOS is turned on, each task will have its own producer client
// and the producer object passed in here will be null. We would then iterate through
// all the active tasks and add their metrics to the output metrics map.
final Collection<StreamsProducer> producers = threadProducer != null ?
Collections.singleton(threadProducer) :
taskProducers.values();
return ClientUtils.producerMetrics(producers);
}
Set<String> producerClientIds() {
if (threadProducer != null) {
return Collections.singleton(getThreadProducerClientId(threadId));
} else {
return taskProducers.keySet()
.stream()
.map(taskId -> getTaskProducerClientId(threadId, taskId))
.collect(Collectors.toSet());
}
}
private LogContext getLogContext(final TaskId taskId) {
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final String logPrefix = threadIdPrefix + String.format("%s [%s] ", "stream-task", taskId);
return new LogContext(logPrefix);
}
public double totalProducerBlockedTime() {
if (threadProducer != null) {
return threadProducer.totalBlockedTime();
}
return taskProducers.values().stream()
.mapToDouble(StreamsProducer::totalBlockedTime)
.sum();
}
}