-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
StreamThread.java
1331 lines (1140 loc) · 57.2 KB
/
StreamThread.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
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.CommitFailedException;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Count;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Rate;
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.LockException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskIdFormatException;
import org.apache.kafka.streams.processor.PartitionGrouper;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.TopologyBuilder;
import org.apache.kafka.streams.state.HostInfo;
import org.apache.kafka.streams.state.internals.ThreadCache;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import static java.util.Collections.singleton;
public class StreamThread extends Thread {
private static final Logger log = LoggerFactory.getLogger(StreamThread.class);
private static final AtomicInteger STREAM_THREAD_ID_SEQUENCE = new AtomicInteger(1);
/**
* Stream thread states are the possible states that a stream thread can be in.
* A thread must only be in one state at a time
* The expected state transitions with the following defined states is:
*
* <pre>
* +-------------+
* | Not Running | <-------+
* +-----+-------+ |
* | |
* v |
* +-----+-------+ |
* +<--- | Running | <----+ |
* | +-----+-------+ | |
* | | | |
* | v | |
* | +-----+-------+ | |
* +<--- | Partitions | | |
* | | Revoked | | |
* | +-----+-------+ | |
* | | | |
* | v | |
* | +-----+-------+ | |
* | | Assigning | | |
* | | Partitions | ---->+ |
* | +-----+-------+ |
* | | |
* | v |
* | +-----+-------+ |
* +---> | Pending | ------->+
* | Shutdown |
* +-------------+
* </pre>
*/
public enum State {
NOT_RUNNING(1), RUNNING(1, 2, 4), PARTITIONS_REVOKED(3, 4), ASSIGNING_PARTITIONS(1, 4), PENDING_SHUTDOWN(0);
private final Set<Integer> validTransitions = new HashSet<>();
State(final Integer... validTransitions) {
this.validTransitions.addAll(Arrays.asList(validTransitions));
}
public boolean isRunning() {
return !this.equals(PENDING_SHUTDOWN) && !this.equals(NOT_RUNNING);
}
public boolean isValidTransition(final State newState) {
return validTransitions.contains(newState.ordinal());
}
}
private volatile State state = State.NOT_RUNNING;
private StateListener stateListener = null;
/**
* Listen to state change events
*/
public interface StateListener {
/**
* Called when state changes
* @param thread thread changing state
* @param newState current state
* @param oldState previous state
*/
void onChange(final StreamThread thread, final State newState, final State oldState);
}
/**
* Set the {@link StateListener} to be notified when state changes. Note this API is internal to
* Kafka Streams and is not intended to be used by an external application.
*/
public void setStateListener(final StateListener listener) {
this.stateListener = listener;
}
/**
* @return The state this instance is in
*/
public synchronized State state() {
return state;
}
private synchronized void setState(State newState) {
State oldState = state;
if (!state.isValidTransition(newState)) {
log.warn("Unexpected state transition from {} to {}.", logPrefix, oldState, newState);
} else {
log.info("{} State transition from {} to {}.", logPrefix, oldState, newState);
}
state = newState;
if (stateListener != null) {
stateListener.onChange(this, state, oldState);
}
}
private synchronized void setStateWhenNotInPendingShutdown(final State newState) {
if (state == State.PENDING_SHUTDOWN) {
return;
}
setState(newState);
}
public final PartitionGrouper partitionGrouper;
private final StreamsMetadataState streamsMetadataState;
public final String applicationId;
public final String clientId;
public final UUID processId;
protected final StreamsConfig config;
protected final TopologyBuilder builder;
protected final Producer<byte[], byte[]> producer;
protected final Consumer<byte[], byte[]> consumer;
protected final Consumer<byte[], byte[]> restoreConsumer;
private final String logPrefix;
private final String threadClientId;
private final Pattern sourceTopicPattern;
private final Map<TaskId, StreamTask> activeTasks;
private final Map<TaskId, StandbyTask> standbyTasks;
private final Map<TopicPartition, StreamTask> activeTasksByPartition;
private final Map<TopicPartition, StandbyTask> standbyTasksByPartition;
private final Set<TaskId> prevActiveTasks;
private final Map<TaskId, StreamTask> suspendedTasks;
private final Map<TaskId, StandbyTask> suspendedStandbyTasks;
private final Time time;
private final int rebalanceTimeoutMs;
private final long pollTimeMs;
private final long cleanTimeMs;
private final long commitTimeMs;
private final StreamsMetricsThreadImpl streamsMetrics;
// TODO: this is not private only for tests, should be better refactored
final StateDirectory stateDirectory;
private String originalReset;
private StreamPartitionAssignor partitionAssignor = null;
private boolean cleanRun = false;
private long timerStartedMs;
private long lastCleanMs;
private long lastCommitMs;
private Throwable rebalanceException = null;
private Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> standbyRecords;
private boolean processStandbyRecords = false;
private ThreadCache cache;
private StoreChangelogReader storeChangelogReader;
private final TaskCreator taskCreator = new TaskCreator();
final ConsumerRebalanceListener rebalanceListener;
public synchronized boolean isInitialized() {
return state == State.RUNNING;
}
public String threadClientId() {
return threadClientId;
}
public StreamThread(TopologyBuilder builder,
StreamsConfig config,
KafkaClientSupplier clientSupplier,
String applicationId,
String clientId,
UUID processId,
Metrics metrics,
Time time,
StreamsMetadataState streamsMetadataState,
final long cacheSizeBytes) {
super(clientId + "-StreamThread-" + STREAM_THREAD_ID_SEQUENCE.getAndIncrement());
this.applicationId = applicationId;
this.config = config;
this.builder = builder;
this.sourceTopicPattern = builder.sourceTopicPattern();
this.clientId = clientId;
this.processId = processId;
this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class);
this.streamsMetadataState = streamsMetadataState;
threadClientId = getName();
this.streamsMetrics = new StreamsMetricsThreadImpl(metrics, "stream-metrics", "thread." + threadClientId,
Collections.singletonMap("client-id", threadClientId));
if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) < 0) {
log.warn("Negative cache size passed in thread [{}]. Reverting to cache size of 0 bytes.", threadClientId);
}
this.cache = new ThreadCache(threadClientId, cacheSizeBytes, this.streamsMetrics);
this.logPrefix = String.format("stream-thread [%s]", threadClientId);
// set the producer and consumer clients
log.info("{} Creating producer client", logPrefix);
this.producer = clientSupplier.getProducer(config.getProducerConfigs(threadClientId));
log.info("{} Creating consumer client", logPrefix);
Map<String, Object> consumerConfigs = config.getConsumerConfigs(this, applicationId, threadClientId);
if (!builder.latestResetTopicsPattern().pattern().equals("") || !builder.earliestResetTopicsPattern().pattern().equals("")) {
originalReset = (String) consumerConfigs.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG);
log.info("{} Custom offset resets specified updating configs original auto offset reset {}", logPrefix, originalReset);
consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
}
this.consumer = clientSupplier.getConsumer(consumerConfigs);
log.info("{} Creating restore consumer client", logPrefix);
this.restoreConsumer = clientSupplier.getRestoreConsumer(config.getRestoreConsumerConfigs(threadClientId));
// initialize the task list
// activeTasks needs to be concurrent as it can be accessed
// by QueryableState
this.activeTasks = new ConcurrentHashMap<>();
this.standbyTasks = new HashMap<>();
this.activeTasksByPartition = new HashMap<>();
this.standbyTasksByPartition = new HashMap<>();
this.prevActiveTasks = new HashSet<>();
this.suspendedTasks = new HashMap<>();
this.suspendedStandbyTasks = new HashMap<>();
// standby ktables
this.standbyRecords = new HashMap<>();
this.stateDirectory = new StateDirectory(applicationId, threadClientId, config.getString(StreamsConfig.STATE_DIR_CONFIG), time);
final Object maxPollInterval = consumerConfigs.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG);
this.rebalanceTimeoutMs = (Integer) ConfigDef.parseType(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, maxPollInterval, Type.INT);
this.pollTimeMs = config.getLong(StreamsConfig.POLL_MS_CONFIG);
this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG);
this.cleanTimeMs = config.getLong(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG);
this.time = time;
this.timerStartedMs = time.milliseconds();
this.lastCleanMs = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment
this.lastCommitMs = timerStartedMs;
this.rebalanceListener = new RebalanceListener(time, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG));
setState(State.RUNNING);
}
public void partitionAssignor(StreamPartitionAssignor partitionAssignor) {
this.partitionAssignor = partitionAssignor;
}
/**
* Execute the stream processors
*
* @throws KafkaException for any Kafka-related exceptions
* @throws Exception for any other non-Kafka exceptions
*/
@Override
public void run() {
log.info("{} Starting", logPrefix);
try {
runLoop();
cleanRun = true;
} catch (KafkaException e) {
// just re-throw the exception as it should be logged already
throw e;
} catch (Exception e) {
// we have caught all Kafka related exceptions, and other runtime exceptions
// should be due to user application errors
log.error("{} Streams application error during processing: ", logPrefix, e);
throw e;
} finally {
shutdown();
}
}
/**
* Shutdown this stream thread.
*/
public synchronized void close() {
log.info("{} Informed thread to shut down", logPrefix);
setState(State.PENDING_SHUTDOWN);
}
public Map<TaskId, StreamTask> tasks() {
return Collections.unmodifiableMap(activeTasks);
}
private void shutdown() {
log.info("{} Shutting down", logPrefix);
shutdownTasksAndState();
// close all embedded clients
try {
producer.close();
} catch (Throwable e) {
log.error("{} Failed to close producer: ", logPrefix, e);
}
try {
consumer.close();
} catch (Throwable e) {
log.error("{} Failed to close consumer: ", logPrefix, e);
}
try {
restoreConsumer.close();
} catch (Throwable e) {
log.error("{} Failed to close restore consumer: ", logPrefix, e);
}
try {
partitionAssignor.close();
} catch (Throwable e) {
log.error("{} Failed to close KafkaStreamClient: ", logPrefix, e);
}
removeStreamTasks();
removeStandbyTasks();
// clean up global tasks
log.info("{} Stream thread shutdown complete", logPrefix);
setState(State.NOT_RUNNING);
streamsMetrics.removeAllSensors();
}
private RuntimeException unAssignChangeLogPartitions() {
try {
// un-assign the change log partitions
restoreConsumer.assign(Collections.<TopicPartition>emptyList());
} catch (RuntimeException e) {
log.error("{} Failed to un-assign change log partitions: ", logPrefix, e);
return e;
}
return null;
}
@SuppressWarnings("ThrowableNotThrown")
private void shutdownTasksAndState() {
log.debug("{} shutdownTasksAndState: shutting down all active tasks {} " +
"and standby tasks {} and suspended tasks {} and suspended standby tasks {}", logPrefix,
activeTasks.keySet(), standbyTasks.keySet(),
suspendedTasks.keySet(), suspendedStandbyTasks.keySet());
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
// Close all processors in topology order
firstException.compareAndSet(null, closeAllTasks());
firstException.compareAndSet(null, closeAllSuspendedTasks());
// flush state
firstException.compareAndSet(null, flushAllState());
// Close all task state managers. Don't need to set exception as all
// state would have been flushed above
closeAllStateManagers(firstException.get() == null);
closeAllSuspendedStateManagers(firstException.get() == null);
// only commit under clean exit
if (cleanRun && firstException.get() == null) {
firstException.set(commitOffsets());
}
// remove the changelog partitions from restore consumer
unAssignChangeLogPartitions();
}
/**
* Similar to shutdownTasksAndState, however does not close the task managers, in the hope that
* soon the tasks will be assigned again
*/
private void suspendTasksAndState() {
log.debug("{} suspendTasksAndState: suspending all active tasks {} and standby tasks {}", logPrefix,
activeTasks.keySet(), standbyTasks.keySet());
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
// Close all topology nodes
firstException.compareAndSet(null, closeAllTasksTopologies());
// flush state
firstException.compareAndSet(null, flushAllState());
// only commit after all state has been flushed and there hasn't been an exception
if (firstException.get() == null) {
// TODO: currently commit failures will not be thrown to users
// while suspending tasks; this need to be re-visit after KIP-98
commitOffsets();
}
// remove the changelog partitions from restore consumer
firstException.compareAndSet(null, unAssignChangeLogPartitions());
updateSuspendedTasks();
if (firstException.get() != null) {
throw new StreamsException(logPrefix + " failed to suspend stream tasks", firstException.get());
}
}
interface AbstractTaskAction {
void apply(final AbstractTask task);
}
private RuntimeException performOnAllTasks(final AbstractTaskAction action,
final String exceptionMessage) {
RuntimeException firstException = null;
final List<AbstractTask> allTasks = new ArrayList<AbstractTask>(activeTasks.values());
allTasks.addAll(standbyTasks.values());
for (final AbstractTask task : allTasks) {
try {
action.apply(task);
} catch (RuntimeException t) {
log.error("{} Failed while executing {} {} due to {}: ",
StreamThread.this.logPrefix,
task.getClass().getSimpleName(),
task.id(),
exceptionMessage,
t);
if (firstException == null) {
firstException = t;
}
}
}
return firstException;
}
private RuntimeException performOnAllSuspendedTasks(final AbstractTaskAction action,
final String exceptionMessage) {
RuntimeException firstException = null;
final List<AbstractTask> allTasks = new ArrayList<AbstractTask>(suspendedTasks.values());
allTasks.addAll(suspendedStandbyTasks.values());
for (final AbstractTask task : allTasks) {
try {
action.apply(task);
} catch (RuntimeException t) {
log.error("{} Failed while executing {} {} due to {}: ",
StreamThread.this.logPrefix,
task.getClass().getSimpleName(),
task.id(),
exceptionMessage,
t);
if (firstException == null) {
firstException = t;
}
}
}
return firstException;
}
private Throwable closeAllStateManagers(final boolean writeCheckpoint) {
return performOnAllTasks(new AbstractTaskAction() {
@Override
public void apply(final AbstractTask task) {
log.info("{} Closing the state manager of task {}", StreamThread.this.logPrefix, task.id());
task.closeStateManager(writeCheckpoint);
}
}, "close state manager");
}
private Throwable closeAllSuspendedStateManagers(final boolean writeCheckpoint) {
return performOnAllSuspendedTasks(new AbstractTaskAction() {
@Override
public void apply(final AbstractTask task) {
log.info("{} Closing the state manager of task {}", StreamThread.this.logPrefix, task.id());
task.closeStateManager(writeCheckpoint);
}
}, "close state manager");
}
private RuntimeException commitOffsets() {
// Exceptions should not prevent this call from going through all shutdown steps
return performOnAllTasks(new AbstractTaskAction() {
@Override
public void apply(final AbstractTask task) {
log.info("{} Committing consumer offsets of task {}", StreamThread.this.logPrefix, task.id());
task.commitOffsets();
}
}, "commit consumer offsets");
}
private RuntimeException flushAllState() {
return performOnAllTasks(new AbstractTaskAction() {
@Override
public void apply(final AbstractTask task) {
log.info("{} Flushing state stores of task {}", StreamThread.this.logPrefix, task.id());
task.flushState();
}
}, "flush state");
}
/**
* Compute the latency based on the current marked timestamp, and update the marked timestamp
* with the current system timestamp.
*
* @return latency
*/
private long computeLatency() {
long previousTimeMs = this.timerStartedMs;
this.timerStartedMs = time.milliseconds();
return Math.max(this.timerStartedMs - previousTimeMs, 0);
}
private void runLoop() {
int totalNumBuffered = 0;
boolean requiresPoll = true;
boolean polledRecords = false;
consumer.subscribe(sourceTopicPattern, rebalanceListener);
while (stillRunning()) {
this.timerStartedMs = time.milliseconds();
// try to fetch some records if necessary
if (requiresPoll) {
requiresPoll = false;
boolean longPoll = totalNumBuffered == 0;
ConsumerRecords<byte[], byte[]> records = null;
try {
records = consumer.poll(longPoll ? this.pollTimeMs : 0);
} catch (NoOffsetForPartitionException ex) {
TopicPartition partition = ex.partition();
if (builder.earliestResetTopicsPattern().matcher(partition.topic()).matches()) {
log.info(String.format("stream-thread [%s] setting topic to consume from earliest offset %s", this.getName(), partition.topic()));
consumer.seekToBeginning(ex.partitions());
} else if (builder.latestResetTopicsPattern().matcher(partition.topic()).matches()) {
consumer.seekToEnd(ex.partitions());
log.info(String.format("stream-thread [%s] setting topic to consume from latest offset %s", this.getName(), partition.topic()));
} else {
if (originalReset == null || (!originalReset.equals("earliest") && !originalReset.equals("latest"))) {
setState(State.PENDING_SHUTDOWN);
String errorMessage = "No valid committed offset found for input topic %s (partition %s) and no valid reset policy configured." +
" You need to set configuration parameter \"auto.offset.reset\" or specify a topic specific reset " +
"policy via KStreamBuilder#stream(StreamsConfig.AutoOffsetReset offsetReset, ...) or KStreamBuilder#table(StreamsConfig.AutoOffsetReset offsetReset, ...)";
throw new StreamsException(String.format(errorMessage, partition.topic(), partition.partition()), ex);
}
if (originalReset.equals("earliest")) {
consumer.seekToBeginning(ex.partitions());
} else if (originalReset.equals("latest")) {
consumer.seekToEnd(ex.partitions());
}
log.info(String.format("stream-thread [%s] no custom setting defined for topic %s using original config %s for offset reset", this.getName(), partition.topic(), originalReset));
}
}
if (rebalanceException != null)
throw new StreamsException(logPrefix + " Failed to rebalance", rebalanceException);
if (records != null && !records.isEmpty()) {
int numAddedRecords = 0;
for (TopicPartition partition : records.partitions()) {
StreamTask task = activeTasksByPartition.get(partition);
if (task == null) {
log.warn("No active tasks for partition " + partition);
continue;
}
numAddedRecords += task.addRecords(partition, records.records(partition));
}
streamsMetrics.skippedRecordsSensor.record(records.count() - numAddedRecords, timerStartedMs);
polledRecords = true;
} else {
polledRecords = false;
}
// only record poll latency is long poll is required
if (longPoll) {
streamsMetrics.pollTimeSensor.record(computeLatency(), timerStartedMs);
}
}
// try to process one fetch record from each task via the topology, and also trigger punctuate
// functions if necessary, which may result in more records going through the topology in this loop
if (totalNumBuffered > 0 || polledRecords) {
totalNumBuffered = 0;
if (!activeTasks.isEmpty()) {
for (StreamTask task : activeTasks.values()) {
totalNumBuffered += task.process();
requiresPoll = requiresPoll || task.requiresPoll();
streamsMetrics.processTimeSensor.record(computeLatency(), timerStartedMs);
maybePunctuate(task);
if (task.commitNeeded())
commitOne(task);
}
} else {
// even when no task is assigned, we must poll to get a task.
requiresPoll = true;
}
} else {
requiresPoll = true;
}
maybeCommit(timerStartedMs);
maybeUpdateStandbyTasks();
maybeClean(timerStartedMs);
}
log.info("{} Shutting down at user request", logPrefix);
}
private void maybeUpdateStandbyTasks() {
if (!standbyTasks.isEmpty()) {
if (processStandbyRecords) {
if (!standbyRecords.isEmpty()) {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> remainingStandbyRecords = new HashMap<>();
for (TopicPartition partition : standbyRecords.keySet()) {
List<ConsumerRecord<byte[], byte[]>> remaining = standbyRecords.get(partition);
if (remaining != null) {
StandbyTask task = standbyTasksByPartition.get(partition);
remaining = task.update(partition, remaining);
if (remaining != null) {
remainingStandbyRecords.put(partition, remaining);
} else {
restoreConsumer.resume(singleton(partition));
}
}
}
standbyRecords = remainingStandbyRecords;
}
processStandbyRecords = false;
}
ConsumerRecords<byte[], byte[]> records = restoreConsumer.poll(0);
if (!records.isEmpty()) {
for (TopicPartition partition : records.partitions()) {
StandbyTask task = standbyTasksByPartition.get(partition);
if (task == null) {
throw new StreamsException(logPrefix + " Missing standby task for partition " + partition);
}
List<ConsumerRecord<byte[], byte[]>> remaining = task.update(partition, records.records(partition));
if (remaining != null) {
restoreConsumer.pause(singleton(partition));
standbyRecords.put(partition, remaining);
}
}
}
}
}
public synchronized boolean stillRunning() {
return state.isRunning();
}
private void maybePunctuate(StreamTask task) {
try {
// check whether we should punctuate based on the task's partition group timestamp;
// which are essentially based on record timestamp.
if (task.maybePunctuate())
streamsMetrics.punctuateTimeSensor.record(computeLatency(), timerStartedMs);
} catch (KafkaException e) {
log.error("{} Failed to punctuate active task {}: ", logPrefix, task.id(), e);
throw e;
}
}
/**
* Commit all tasks owned by this thread if specified interval time has elapsed
*/
protected void maybeCommit(final long now) {
if (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) {
log.info("{} Committing all active tasks {} and standby tasks {} because the commit interval {}ms has elapsed",
logPrefix, commitTimeMs, activeTasks, standbyTasks);
commitAll();
lastCommitMs = now;
processStandbyRecords = true;
}
}
/**
* Cleanup any states of the tasks that have been removed from this thread
*/
protected void maybeClean(final long now) {
if (now > lastCleanMs + cleanTimeMs) {
stateDirectory.cleanRemovedTasks(cleanTimeMs);
lastCleanMs = now;
}
}
/**
* Commit the states of all its tasks
*/
private void commitAll() {
for (StreamTask task : activeTasks.values()) {
commitOne(task);
}
for (StandbyTask task : standbyTasks.values()) {
commitOne(task);
}
}
/**
* Commit the state of a task
*/
private void commitOne(AbstractTask task) {
log.info("{} Committing task {} {}", logPrefix, task.getClass().getSimpleName(), task.id());
try {
task.commit();
} catch (CommitFailedException e) {
// commit failed. Just log it.
log.warn("{} Failed to commit {} {} state: ", logPrefix, task.getClass().getSimpleName(), task.id(), e);
} catch (KafkaException e) {
// commit failed due to an unexpected exception. Log it and rethrow the exception.
log.error("{} Failed to commit {} {} state: ", logPrefix, task.getClass().getSimpleName(), task.id(), e);
throw e;
}
streamsMetrics.commitTimeSensor.record(computeLatency(), timerStartedMs);
}
/**
* Returns ids of tasks that were being executed before the rebalance.
*/
public Set<TaskId> prevActiveTasks() {
return Collections.unmodifiableSet(prevActiveTasks);
}
/**
* Returns ids of tasks whose states are kept on the local storage.
*/
public Set<TaskId> cachedTasks() {
// A client could contain some inactive tasks whose states are still kept on the local storage in the following scenarios:
// 1) the client is actively maintaining standby tasks by maintaining their states from the change log.
// 2) the client has just got some tasks migrated out of itself to other clients while these task states
// have not been cleaned up yet (this can happen in a rolling bounce upgrade, for example).
HashSet<TaskId> tasks = new HashSet<>();
File[] stateDirs = stateDirectory.listTaskDirectories();
if (stateDirs != null) {
for (File dir : stateDirs) {
try {
TaskId id = TaskId.parse(dir.getName());
// if the checkpoint file exists, the state is valid.
if (new File(dir, ProcessorStateManager.CHECKPOINT_FILE_NAME).exists())
tasks.add(id);
} catch (TaskIdFormatException e) {
// there may be some unknown files that sits in the same directory,
// we should ignore these files instead trying to delete them as well
}
}
}
return tasks;
}
private StreamTask findMatchingSuspendedTask(final TaskId taskId, final Set<TopicPartition> partitions) {
if (suspendedTasks.containsKey(taskId)) {
final StreamTask task = suspendedTasks.get(taskId);
if (task.partitions.equals(partitions)) {
return task;
}
}
return null;
}
private StandbyTask findMatchingSuspendedStandbyTask(final TaskId taskId, final Set<TopicPartition> partitions) {
if (suspendedStandbyTasks.containsKey(taskId)) {
final StandbyTask task = suspendedStandbyTasks.get(taskId);
if (task.partitions.equals(partitions)) {
return task;
}
}
return null;
}
private void closeNonAssignedSuspendedTasks() {
final Map<TaskId, Set<TopicPartition>> newTaskAssignment = partitionAssignor.activeTasks();
final Iterator<Map.Entry<TaskId, StreamTask>> suspendedTaskIterator = suspendedTasks.entrySet().iterator();
while (suspendedTaskIterator.hasNext()) {
final Map.Entry<TaskId, StreamTask> next = suspendedTaskIterator.next();
final StreamTask task = next.getValue();
final Set<TopicPartition> assignedPartitionsForTask = newTaskAssignment.get(next.getKey());
if (!task.partitions().equals(assignedPartitionsForTask)) {
log.debug("{} Closing suspended non-assigned active task {}", logPrefix, task.id);
try {
task.close();
task.closeStateManager(true);
} catch (Exception e) {
log.error("{} Failed to remove suspended task {}", logPrefix, next.getKey(), e);
} finally {
suspendedTaskIterator.remove();
}
}
}
}
private void closeNonAssignedSuspendedStandbyTasks() {
final Set<TaskId> currentSuspendedTaskIds = partitionAssignor.standbyTasks().keySet();
final Iterator<Map.Entry<TaskId, StandbyTask>> standByTaskIterator = suspendedStandbyTasks.entrySet().iterator();
while (standByTaskIterator.hasNext()) {
final Map.Entry<TaskId, StandbyTask> suspendedTask = standByTaskIterator.next();
if (!currentSuspendedTaskIds.contains(suspendedTask.getKey())) {
final StandbyTask task = suspendedTask.getValue();
log.debug("{} Closing suspended non-assigned standby task {}", logPrefix, task.id);
try {
task.close();
task.closeStateManager(true);
} catch (Exception e) {
log.error("{} Failed to remove suspended standby task {}", logPrefix, task.id, e);
} finally {
standByTaskIterator.remove();
}
}
}
}
protected StreamTask createStreamTask(TaskId id, Collection<TopicPartition> partitions) {
log.debug("{} Creating new active task {} with assigned partitions {}", logPrefix, id, partitions);
streamsMetrics.taskCreatedSensor.record();
final ProcessorTopology topology = builder.build(id.topicGroupId);
final RecordCollector recordCollector = new RecordCollectorImpl(producer, id.toString());
try {
return new StreamTask(id, applicationId, partitions, topology, consumer, storeChangelogReader, config, streamsMetrics, stateDirectory, cache, time, recordCollector);
} finally {
log.info("{} Created active task {} with assigned partitions {}", logPrefix, id, partitions);
}
}
private void addStreamTasks(Collection<TopicPartition> assignment, final long start) {
if (partitionAssignor == null)
throw new IllegalStateException(logPrefix + " Partition assignor has not been initialized while adding stream tasks: this should not happen.");
final Map<TaskId, Set<TopicPartition>> newTasks = new HashMap<>();
// collect newly assigned tasks and reopen re-assigned tasks
for (Map.Entry<TaskId, Set<TopicPartition>> entry : partitionAssignor.activeTasks().entrySet()) {
final TaskId taskId = entry.getKey();
final Set<TopicPartition> partitions = entry.getValue();
if (assignment.containsAll(partitions)) {
try {
StreamTask task = findMatchingSuspendedTask(taskId, partitions);
if (task != null) {
log.debug("{} recycling old task {}", logPrefix, taskId);
suspendedTasks.remove(taskId);
task.initTopology();
activeTasks.put(taskId, task);
for (TopicPartition partition : partitions) {
activeTasksByPartition.put(partition, task);
}
} else {
newTasks.put(taskId, partitions);
}
} catch (StreamsException e) {
log.error("{} Failed to create an active task {}: ", logPrefix, taskId, e);
throw e;
}
} else {
log.warn("{} Task {} owned partitions {} are not contained in the assignment {}", logPrefix, taskId, partitions, assignment);
}
}
// create all newly assigned tasks (guard against race condition with other thread via backoff and retry)
// -> other thread will call removeSuspendedTasks(); eventually
log.debug("{} new active tasks to be created: {}", logPrefix, newTasks);
taskCreator.retryWithBackoff(newTasks, start);
}
private StandbyTask createStandbyTask(TaskId id, Collection<TopicPartition> partitions) {
log.debug("{} Creating new standby task {} with assigned partitions {}", logPrefix, id, partitions);
streamsMetrics.taskCreatedSensor.record();
ProcessorTopology topology = builder.build(id.topicGroupId);
if (!topology.stateStores().isEmpty()) {
try {
return new StandbyTask(id, applicationId, partitions, topology, consumer, storeChangelogReader, config, streamsMetrics, stateDirectory);
} finally {
log.info("{} Created standby task {} with assigned partitions {}", logPrefix, id, partitions);
}
} else {
log.info("{} Skipped standby task {} with assigned partitions {} since it does not have any state stores to materialize", logPrefix, id, partitions);
return null;
}
}
private void addStandbyTasks(final long start) {
if (partitionAssignor == null)
throw new IllegalStateException(logPrefix + " Partition assignor has not been initialized while adding standby tasks: this should not happen.");
Map<TopicPartition, Long> checkpointedOffsets = new HashMap<>();
final Map<TaskId, Set<TopicPartition>> newStandbyTasks = new HashMap<>();
// collect newly assigned standby tasks and reopen re-assigned standby tasks
for (Map.Entry<TaskId, Set<TopicPartition>> entry : partitionAssignor.standbyTasks().entrySet()) {
final TaskId taskId = entry.getKey();
final Set<TopicPartition> partitions = entry.getValue();
StandbyTask task = findMatchingSuspendedStandbyTask(taskId, partitions);
if (task != null) {
log.debug("{} recycling old standby task {}", logPrefix, taskId);
suspendedStandbyTasks.remove(taskId);
task.initTopology();
} else {
newStandbyTasks.put(taskId, partitions);
}
updateStandByTaskMaps(checkpointedOffsets, taskId, partitions, task);
}
// create all newly assigned standby tasks (guard against race condition with other thread via backoff and retry)
// -> other thread will call removeSuspendedStandbyTasks(); eventually
log.debug("{} new standby tasks to be created: {}", logPrefix, newStandbyTasks);
new StandbyTaskCreator(checkpointedOffsets).retryWithBackoff(newStandbyTasks, start);
restoreConsumer.assign(new ArrayList<>(checkpointedOffsets.keySet()));
for (Map.Entry<TopicPartition, Long> entry : checkpointedOffsets.entrySet()) {
TopicPartition partition = entry.getKey();
long offset = entry.getValue();
if (offset >= 0) {