/
GlobalStateManagerImpl.java
407 lines (366 loc) · 18.3 KB
/
GlobalStateManagerImpl.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
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
/*
* 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.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.InvalidOffsetException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.utils.FixedOrderMap;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateRestoreListener;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.streams.state.internals.RecordConverter;
import org.slf4j.Logger;
import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.converterForStore;
/**
* This class is responsible for the initialization, restoration, closing, flushing etc
* of Global State Stores. There is only ever 1 instance of this class per Application Instance.
*/
public class GlobalStateManagerImpl implements GlobalStateManager {
private final Logger log;
private final boolean eosEnabled;
private final ProcessorTopology topology;
private final Consumer<byte[], byte[]> globalConsumer;
private final File baseDir;
private final StateDirectory stateDirectory;
private final Set<String> globalStoreNames = new HashSet<>();
private final FixedOrderMap<String, Optional<StateStore>> globalStores = new FixedOrderMap<>();
private final StateRestoreListener stateRestoreListener;
private InternalProcessorContext globalProcessorContext;
private final int retries;
private final long retryBackoffMs;
private final Duration pollTime;
private final Set<String> globalNonPersistentStoresTopics = new HashSet<>();
private final OffsetCheckpoint checkpointFile;
private final Map<TopicPartition, Long> checkpointFileCache;
public GlobalStateManagerImpl(final LogContext logContext,
final ProcessorTopology topology,
final Consumer<byte[], byte[]> globalConsumer,
final StateDirectory stateDirectory,
final StateRestoreListener stateRestoreListener,
final StreamsConfig config) {
eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
baseDir = stateDirectory.globalStateDir();
checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME));
checkpointFileCache = new HashMap<>();
// Find non persistent store's topics
final Map<String, String> storeToChangelogTopic = topology.storeToChangelogTopic();
for (final StateStore store : topology.globalStateStores()) {
if (!store.persistent()) {
globalNonPersistentStoresTopics.add(storeToChangelogTopic.get(store.name()));
}
}
log = logContext.logger(GlobalStateManagerImpl.class);
this.topology = topology;
this.globalConsumer = globalConsumer;
this.stateDirectory = stateDirectory;
this.stateRestoreListener = stateRestoreListener;
retries = config.getInt(StreamsConfig.RETRIES_CONFIG);
retryBackoffMs = config.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG);
pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG));
}
@Override
public void setGlobalProcessorContext(final InternalProcessorContext globalProcessorContext) {
this.globalProcessorContext = globalProcessorContext;
}
@Override
public Set<String> initialize() {
try {
if (!stateDirectory.lockGlobalState()) {
throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir));
}
} catch (final IOException e) {
throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir), e);
}
try {
checkpointFileCache.putAll(checkpointFile.read());
} catch (final IOException e) {
try {
stateDirectory.unlockGlobalState();
} catch (final IOException e1) {
log.error("Failed to unlock the global state directory", e);
}
throw new StreamsException("Failed to read checkpoints for global state globalStores", e);
}
final List<StateStore> stateStores = topology.globalStateStores();
for (final StateStore stateStore : stateStores) {
globalStoreNames.add(stateStore.name());
stateStore.init(globalProcessorContext, stateStore);
}
return Collections.unmodifiableSet(globalStoreNames);
}
@Override
public void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext) {
StateManagerUtil.reinitializeStateStoresForPartitions(
log,
eosEnabled,
baseDir,
globalStores,
topology.storeToChangelogTopic(),
partitions,
processorContext,
checkpointFile,
checkpointFileCache
);
globalConsumer.assign(partitions);
globalConsumer.seekToBeginning(partitions);
}
@Override
public StateStore getGlobalStore(final String name) {
return globalStores.getOrDefault(name, Optional.empty()).orElse(null);
}
@Override
public StateStore getStore(final String name) {
return getGlobalStore(name);
}
public File baseDir() {
return baseDir;
}
public void register(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {
if (globalStores.containsKey(store.name())) {
throw new IllegalArgumentException(String.format("Global Store %s has already been registered", store.name()));
}
if (!globalStoreNames.contains(store.name())) {
throw new IllegalArgumentException(String.format("Trying to register store %s that is not a known global store", store.name()));
}
if (stateRestoreCallback == null) {
throw new IllegalArgumentException(String.format("The stateRestoreCallback provided for store %s was null", store.name()));
}
log.info("Restoring state for global store {}", store.name());
final List<TopicPartition> topicPartitions = topicPartitionsForStore(store);
Map<TopicPartition, Long> highWatermarks = null;
int attempts = 0;
while (highWatermarks == null) {
try {
highWatermarks = globalConsumer.endOffsets(topicPartitions);
} catch (final TimeoutException retryableException) {
if (++attempts > retries) {
log.error("Failed to get end offsets for topic partitions of global store {} after {} retry attempts. " +
"You can increase the number of retries via configuration parameter `retries`.",
store.name(),
retries,
retryableException);
throw new StreamsException(String.format("Failed to get end offsets for topic partitions of global store %s after %d retry attempts. " +
"You can increase the number of retries via configuration parameter `retries`.", store.name(), retries),
retryableException);
}
log.debug("Failed to get end offsets for partitions {}, backing off for {} ms to retry (attempt {} of {})",
topicPartitions,
retryBackoffMs,
attempts,
retries,
retryableException);
Utils.sleep(retryBackoffMs);
}
}
try {
restoreState(
stateRestoreCallback,
topicPartitions,
highWatermarks,
store.name(),
converterForStore(store)
);
globalStores.put(store.name(), Optional.of(store));
} finally {
globalConsumer.unsubscribe();
}
}
private List<TopicPartition> topicPartitionsForStore(final StateStore store) {
final String sourceTopic = topology.storeToChangelogTopic().get(store.name());
List<PartitionInfo> partitionInfos;
int attempts = 0;
while (true) {
try {
partitionInfos = globalConsumer.partitionsFor(sourceTopic);
break;
} catch (final TimeoutException retryableException) {
if (++attempts > retries) {
log.error("Failed to get partitions for topic {} after {} retry attempts due to timeout. " +
"The broker may be transiently unavailable at the moment. " +
"You can increase the number of retries via configuration parameter `retries`.",
sourceTopic,
retries,
retryableException);
throw new StreamsException(String.format("Failed to get partitions for topic %s after %d retry attempts due to timeout. " +
"The broker may be transiently unavailable at the moment. " +
"You can increase the number of retries via configuration parameter `retries`.", sourceTopic, retries),
retryableException);
}
log.debug("Failed to get partitions for topic {} due to timeout. The broker may be transiently unavailable at the moment. " +
"Backing off for {} ms to retry (attempt {} of {})",
sourceTopic,
retryBackoffMs,
attempts,
retries,
retryableException);
Utils.sleep(retryBackoffMs);
}
}
if (partitionInfos == null || partitionInfos.isEmpty()) {
throw new StreamsException(String.format("There are no partitions available for topic %s when initializing global store %s", sourceTopic, store.name()));
}
final List<TopicPartition> topicPartitions = new ArrayList<>();
for (final PartitionInfo partition : partitionInfos) {
topicPartitions.add(new TopicPartition(partition.topic(), partition.partition()));
}
return topicPartitions;
}
private void restoreState(final StateRestoreCallback stateRestoreCallback,
final List<TopicPartition> topicPartitions,
final Map<TopicPartition, Long> highWatermarks,
final String storeName,
final RecordConverter recordConverter) {
for (final TopicPartition topicPartition : topicPartitions) {
globalConsumer.assign(Collections.singletonList(topicPartition));
final Long checkpoint = checkpointFileCache.get(topicPartition);
if (checkpoint != null) {
globalConsumer.seek(topicPartition, checkpoint);
} else {
globalConsumer.seekToBeginning(Collections.singletonList(topicPartition));
}
long offset = globalConsumer.position(topicPartition);
final Long highWatermark = highWatermarks.get(topicPartition);
final RecordBatchingStateRestoreCallback stateRestoreAdapter =
StateRestoreCallbackAdapter.adapt(stateRestoreCallback);
stateRestoreListener.onRestoreStart(topicPartition, storeName, offset, highWatermark);
long restoreCount = 0L;
while (offset < highWatermark) {
try {
final ConsumerRecords<byte[], byte[]> records = globalConsumer.poll(pollTime);
final List<ConsumerRecord<byte[], byte[]>> restoreRecords = new ArrayList<>();
for (final ConsumerRecord<byte[], byte[]> record : records.records(topicPartition)) {
if (record.key() != null) {
restoreRecords.add(recordConverter.convert(record));
}
}
offset = globalConsumer.position(topicPartition);
stateRestoreAdapter.restoreBatch(restoreRecords);
stateRestoreListener.onBatchRestored(topicPartition, storeName, offset, restoreRecords.size());
restoreCount += restoreRecords.size();
} catch (final InvalidOffsetException recoverableException) {
log.warn("Restoring GlobalStore {} failed due to: {}. Deleting global store to recreate from scratch.",
storeName,
recoverableException.toString());
reinitializeStateStoresForPartitions(recoverableException.partitions(), globalProcessorContext);
stateRestoreListener.onRestoreStart(topicPartition, storeName, offset, highWatermark);
restoreCount = 0L;
}
}
stateRestoreListener.onRestoreEnd(topicPartition, storeName, restoreCount);
checkpointFileCache.put(topicPartition, offset);
}
}
@Override
public void flush() {
log.debug("Flushing all global globalStores registered in the state manager");
for (final Map.Entry<String, Optional<StateStore>> entry : globalStores.entrySet()) {
if (entry.getValue().isPresent()) {
final StateStore store = entry.getValue().get();
try {
log.trace("Flushing global store={}", store.name());
store.flush();
} catch (final RuntimeException e) {
throw new ProcessorStateException(
String.format("Failed to flush global state store %s", store.name()),
e
);
}
} else {
throw new IllegalStateException("Expected " + entry.getKey() + " to have been initialized");
}
}
}
@Override
public void close(final boolean clean) throws IOException {
try {
if (globalStores.isEmpty()) {
return;
}
final StringBuilder closeFailed = new StringBuilder();
for (final Map.Entry<String, Optional<StateStore>> entry : globalStores.entrySet()) {
if (entry.getValue().isPresent()) {
log.debug("Closing global storage engine {}", entry.getKey());
try {
entry.getValue().get().close();
} catch (final RuntimeException e) {
log.error("Failed to close global state store {}", entry.getKey(), e);
closeFailed.append("Failed to close global state store:")
.append(entry.getKey())
.append(". Reason: ")
.append(e)
.append("\n");
}
globalStores.put(entry.getKey(), Optional.empty());
} else {
log.info("Skipping to close non-initialized store {}", entry.getKey());
}
}
if (closeFailed.length() > 0) {
throw new ProcessorStateException("Exceptions caught during close of 1 or more global state globalStores\n" + closeFailed);
}
} finally {
stateDirectory.unlockGlobalState();
}
}
@Override
public void checkpoint(final Map<TopicPartition, Long> offsets) {
checkpointFileCache.putAll(offsets);
final Map<TopicPartition, Long> filteredOffsets = new HashMap<>();
// Skip non persistent store
for (final Map.Entry<TopicPartition, Long> topicPartitionOffset : checkpointFileCache.entrySet()) {
final String topic = topicPartitionOffset.getKey().topic();
if (!globalNonPersistentStoresTopics.contains(topic)) {
filteredOffsets.put(topicPartitionOffset.getKey(), topicPartitionOffset.getValue());
}
}
try {
checkpointFile.write(filteredOffsets);
} catch (final IOException e) {
log.warn("Failed to write offset checkpoint file to {} for global stores: {}", checkpointFile, e);
}
}
@Override
public Map<TopicPartition, Long> checkpointed() {
return Collections.unmodifiableMap(checkpointFileCache);
}
}