/
AsyncKafkaConsumer.java
1927 lines (1713 loc) · 84.7 KB
/
AsyncKafkaConsumer.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.clients.consumer.internals;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.ClientUtils;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.GroupRebalanceConfig;
import org.apache.kafka.clients.KafkaClient;
import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.ConsumerInterceptor;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.GroupProtocol;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent;
import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent;
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent;
import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
import org.apache.kafka.clients.consumer.internals.events.PollApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent;
import org.apache.kafka.clients.consumer.internals.metrics.KafkaConsumerMetrics;
import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.InvalidGroupIdException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.ListOffsetsRequest;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter;
import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils;
import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.event.Level;
import java.net.InetSocketAddress;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.ConcurrentModificationException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static java.util.Objects.requireNonNull;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.refreshCommittedOffsets;
import static org.apache.kafka.common.utils.Utils.closeQuietly;
import static org.apache.kafka.common.utils.Utils.isBlank;
import static org.apache.kafka.common.utils.Utils.join;
import static org.apache.kafka.common.utils.Utils.swallow;
/**
* This {@link Consumer} implementation uses an {@link ApplicationEventHandler event handler} to process
* {@link ApplicationEvent application events} so that the network I/O can be processed in a dedicated
* {@link ConsumerNetworkThread network thread}. Visit
* <a href="https://cwiki.apache.org/confluence/display/KAFKA/Consumer+threading+refactor+design">this document</a>
* for implementation detail.
*
* <p/>
*
* <em>Note:</em> this {@link Consumer} implementation is part of the revised consumer group protocol from KIP-848.
* This class should not be invoked directly; users should instead create a {@link KafkaConsumer} as before.
* This consumer implements the new consumer group protocol and is intended to be the default in coming releases.
*/
public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
private static final long NO_CURRENT_THREAD = -1L;
/**
* An {@link org.apache.kafka.clients.consumer.internals.events.EventProcessor} that is created and executes in the
* application thread for the purpose of processing {@link BackgroundEvent background events} generated by the
* {@link ConsumerNetworkThread network thread}.
* Those events are generally of two types:
*
* <ul>
* <li>Errors that occur in the network thread that need to be propagated to the application thread</li>
* <li>{@link ConsumerRebalanceListener} callbacks that are to be executed on the application thread</li>
* </ul>
*/
private class BackgroundEventProcessor extends EventProcessor<BackgroundEvent> {
private final ApplicationEventHandler applicationEventHandler;
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
public BackgroundEventProcessor(final LogContext logContext,
final BlockingQueue<BackgroundEvent> backgroundEventQueue,
final ApplicationEventHandler applicationEventHandler,
final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) {
super(logContext, backgroundEventQueue);
this.applicationEventHandler = applicationEventHandler;
this.rebalanceListenerInvoker = rebalanceListenerInvoker;
}
/**
* Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}.
* It is possible that {@link org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent an error}
* could occur when processing the events. In such cases, the processor will take a reference to the first
* error, continue to process the remaining events, and then throw the first error that occurred.
*/
@Override
public boolean process() {
AtomicReference<KafkaException> firstError = new AtomicReference<>();
ProcessHandler<BackgroundEvent> processHandler = (event, error) -> {
if (error.isPresent()) {
KafkaException e = error.get();
if (!firstError.compareAndSet(null, e)) {
log.warn("An error occurred when processing the event: {}", e.getMessage(), e);
}
}
};
boolean hadEvents = process(processHandler);
if (firstError.get() != null)
throw firstError.get();
return hadEvents;
}
@Override
public void process(final BackgroundEvent event) {
switch (event.type()) {
case ERROR:
process((ErrorBackgroundEvent) event);
break;
case GROUP_METADATA_UPDATE:
process((GroupMetadataUpdateEvent) event);
break;
case CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED:
process((ConsumerRebalanceListenerCallbackNeededEvent) event);
break;
default:
throw new IllegalArgumentException("Background event type " + event.type() + " was not expected");
}
}
private void process(final ErrorBackgroundEvent event) {
throw event.error();
}
private void process(final GroupMetadataUpdateEvent event) {
if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) {
final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get();
AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata(
currentGroupMetadata.groupId(),
event.memberEpoch(),
event.memberId(),
currentGroupMetadata.groupInstanceId()
));
}
}
private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
ApplicationEvent invokedEvent = invokeRebalanceCallbacks(
rebalanceListenerInvoker,
event.methodName(),
event.partitions(),
event.future()
);
applicationEventHandler.add(invokedEvent);
}
}
private final ApplicationEventHandler applicationEventHandler;
private final Time time;
private Optional<ConsumerGroupMetadata> groupMetadata = Optional.empty();
private final KafkaConsumerMetrics kafkaConsumerMetrics;
private Logger log;
private final String clientId;
private final BackgroundEventProcessor backgroundEventProcessor;
private final Deserializers<K, V> deserializers;
/**
* A thread-safe {@link FetchBuffer fetch buffer} for the results that are populated in the
* {@link ConsumerNetworkThread network thread} when the results are available. Because of the interaction
* of the fetch buffer in the application thread and the network I/O thread, this is shared between the
* two threads and is thus designed to be thread-safe.
*/
private final FetchBuffer fetchBuffer;
private final FetchCollector<K, V> fetchCollector;
private final ConsumerInterceptors<K, V> interceptors;
private final IsolationLevel isolationLevel;
private final SubscriptionState subscriptions;
private final ConsumerMetadata metadata;
private final Metrics metrics;
private final long retryBackoffMs;
private final int defaultApiTimeoutMs;
private final boolean autoCommitEnabled;
private volatile boolean closed = false;
private final List<ConsumerPartitionAssignor> assignors;
private final Optional<ClientTelemetryReporter> clientTelemetryReporter;
// to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates
private boolean cachedSubscriptionHasAllFetchPositions;
private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
// currentThread holds the threadId of the current thread accessing the AsyncKafkaConsumer
// and is used to prevent multithreaded access
private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
private final AtomicInteger refCount = new AtomicInteger(0);
AsyncKafkaConsumer(final ConsumerConfig config,
final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer) {
this(
config,
keyDeserializer,
valueDeserializer,
Time.SYSTEM,
ApplicationEventHandler::new,
FetchCollector::new,
ConsumerMetadata::new,
new LinkedBlockingQueue<>()
);
}
// Visible for testing
AsyncKafkaConsumer(final ConsumerConfig config,
final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer,
final Time time,
final ApplicationEventHandlerFactory applicationEventHandlerFactory,
final FetchCollectorFactory<K, V> fetchCollectorFactory,
final ConsumerMetadataFactory metadataFactory,
final LinkedBlockingQueue<BackgroundEvent> backgroundEventQueue) {
try {
GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
GroupRebalanceConfig.ProtocolType.CONSUMER
);
this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
LogContext logContext = createLogContext(config, groupRebalanceConfig);
this.log = logContext.logger(getClass());
log.debug("Initializing the Kafka consumer");
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.time = time;
List<MetricsReporter> reporters = CommonClientConfigs.metricsReporters(clientId, config);
this.clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config);
this.clientTelemetryReporter.ifPresent(reporters::add);
this.metrics = createMetrics(config, time, reporters);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
List<ConsumerInterceptor<K, V>> interceptorList = configuredConsumerInterceptors(config);
this.interceptors = new ConsumerInterceptors<>(interceptorList);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer);
this.subscriptions = createSubscriptionState(config, logContext);
ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(),
interceptorList,
Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer));
this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners);
final List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config);
metadata.bootstrap(addresses);
FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics);
FetchConfig fetchConfig = new FetchConfig(config);
this.isolationLevel = fetchConfig.isolationLevel;
ApiVersions apiVersions = new ApiVersions();
final BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(
logContext,
backgroundEventQueue
);
// This FetchBuffer is shared between the application and network threads.
this.fetchBuffer = new FetchBuffer(logContext);
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier = NetworkClientDelegate.supplier(time,
logContext,
metadata,
config,
apiVersions,
metrics,
fetchMetricsManager,
clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null));
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
final Supplier<RequestManagers> requestManagersSupplier = RequestManagers.supplier(time,
logContext,
backgroundEventHandler,
metadata,
subscriptions,
fetchBuffer,
config,
groupRebalanceConfig,
apiVersions,
fetchMetricsManager,
networkClientDelegateSupplier,
clientTelemetryReporter,
metrics,
offsetCommitCallbackInvoker);
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
metadata,
applicationEventQueue,
requestManagersSupplier);
this.applicationEventHandler = applicationEventHandlerFactory.build(
logContext,
time,
applicationEventQueue,
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
time,
new RebalanceCallbackMetricsManager(metrics)
);
this.backgroundEventProcessor = new BackgroundEventProcessor(
logContext,
backgroundEventQueue,
applicationEventHandler,
rebalanceListenerInvoker
);
this.assignors = ConsumerPartitionAssignor.getAssignorInstances(
config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId))
);
this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig);
// The FetchCollector is only used on the application thread.
this.fetchCollector = fetchCollectorFactory.build(logContext,
metadata,
subscriptions,
fetchConfig,
deserializers,
fetchMetricsManager,
time);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
if (groupMetadata.isPresent() &&
GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) {
config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread
}
config.logUnused();
AppInfoParser.registerAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics, time.milliseconds());
log.debug("Kafka consumer initialized");
} catch (Throwable t) {
// call close methods if internal objects are already constructed; this is to prevent resource leak. see KAFKA-2121
// we do not need to call `close` at all when `log` is null, which means no internal objects were initialized.
if (this.log != null) {
close(Duration.ZERO, true);
}
// now propagate the exception
throw new KafkaException("Failed to construct kafka consumer", t);
}
}
// Visible for testing
AsyncKafkaConsumer(LogContext logContext,
String clientId,
Deserializers<K, V> deserializers,
FetchBuffer fetchBuffer,
FetchCollector<K, V> fetchCollector,
ConsumerInterceptors<K, V> interceptors,
Time time,
ApplicationEventHandler applicationEventHandler,
BlockingQueue<BackgroundEvent> backgroundEventQueue,
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker,
Metrics metrics,
SubscriptionState subscriptions,
ConsumerMetadata metadata,
long retryBackoffMs,
int defaultApiTimeoutMs,
List<ConsumerPartitionAssignor> assignors,
String groupId,
boolean autoCommitEnabled) {
this.log = logContext.logger(getClass());
this.subscriptions = subscriptions;
this.clientId = clientId;
this.fetchBuffer = fetchBuffer;
this.fetchCollector = fetchCollector;
this.isolationLevel = IsolationLevel.READ_UNCOMMITTED;
this.interceptors = Objects.requireNonNull(interceptors);
this.time = time;
this.backgroundEventProcessor = new BackgroundEventProcessor(
logContext,
backgroundEventQueue,
applicationEventHandler,
rebalanceListenerInvoker
);
this.metrics = metrics;
this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty());
this.metadata = metadata;
this.retryBackoffMs = retryBackoffMs;
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.deserializers = deserializers;
this.applicationEventHandler = applicationEventHandler;
this.assignors = assignors;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.clientTelemetryReporter = Optional.empty();
this.autoCommitEnabled = autoCommitEnabled;
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
}
AsyncKafkaConsumer(LogContext logContext,
Time time,
ConsumerConfig config,
Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer,
KafkaClient client,
SubscriptionState subscriptions,
ConsumerMetadata metadata,
List<ConsumerPartitionAssignor> assignors) {
this.log = logContext.logger(getClass());
this.subscriptions = subscriptions;
this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
this.fetchBuffer = new FetchBuffer(logContext);
this.isolationLevel = IsolationLevel.READ_UNCOMMITTED;
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList());
this.time = time;
this.metrics = new Metrics(time);
this.metadata = metadata;
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
this.assignors = assignors;
this.clientTelemetryReporter = Optional.empty();
ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX);
FetchMetricsManager fetchMetricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics);
this.fetchCollector = new FetchCollector<>(logContext,
metadata,
subscriptions,
new FetchConfig(config),
deserializers,
fetchMetricsManager,
time);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
GroupRebalanceConfig.ProtocolType.CONSUMER
);
this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig);
BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
BlockingQueue<BackgroundEvent> backgroundEventQueue = new LinkedBlockingQueue<>();
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(
logContext,
backgroundEventQueue
);
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
time,
new RebalanceCallbackMetricsManager(metrics)
);
ApiVersions apiVersions = new ApiVersions();
Supplier<NetworkClientDelegate> networkClientDelegateSupplier = () -> new NetworkClientDelegate(
time,
config,
logContext,
client
);
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
Supplier<RequestManagers> requestManagersSupplier = RequestManagers.supplier(
time,
logContext,
backgroundEventHandler,
metadata,
subscriptions,
fetchBuffer,
config,
groupRebalanceConfig,
apiVersions,
fetchMetricsManager,
networkClientDelegateSupplier,
clientTelemetryReporter,
metrics,
offsetCommitCallbackInvoker
);
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
logContext,
metadata,
applicationEventQueue,
requestManagersSupplier
);
this.applicationEventHandler = new ApplicationEventHandler(logContext,
time,
applicationEventQueue,
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
this.backgroundEventProcessor = new BackgroundEventProcessor(
logContext,
backgroundEventQueue,
applicationEventHandler,
rebalanceListenerInvoker
);
}
// auxiliary interface for testing
interface ApplicationEventHandlerFactory {
ApplicationEventHandler build(
final LogContext logContext,
final Time time,
final BlockingQueue<ApplicationEvent> applicationEventQueue,
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
final Supplier<RequestManagers> requestManagersSupplier
);
}
// auxiliary interface for testing
interface FetchCollectorFactory<K, V> {
FetchCollector<K, V> build(
final LogContext logContext,
final ConsumerMetadata metadata,
final SubscriptionState subscriptions,
final FetchConfig fetchConfig,
final Deserializers<K, V> deserializers,
final FetchMetricsManager metricsManager,
final Time time
);
}
// auxiliary interface for testing
interface ConsumerMetadataFactory {
ConsumerMetadata build(
final ConsumerConfig config,
final SubscriptionState subscriptions,
final LogContext logContext,
final ClusterResourceListeners clusterResourceListeners
);
}
private Optional<ConsumerGroupMetadata> initializeGroupMetadata(final ConsumerConfig config,
final GroupRebalanceConfig groupRebalanceConfig) {
final Optional<ConsumerGroupMetadata> groupMetadata = initializeGroupMetadata(
groupRebalanceConfig.groupId,
groupRebalanceConfig.groupInstanceId
);
if (!groupMetadata.isPresent()) {
config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
config.ignore(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED);
}
return groupMetadata;
}
private Optional<ConsumerGroupMetadata> initializeGroupMetadata(final String groupId,
final Optional<String> groupInstanceId) {
if (groupId != null) {
if (groupId.isEmpty()) {
throw new InvalidGroupIdException("The configured " + ConsumerConfig.GROUP_ID_CONFIG
+ " should not be an empty string or whitespace.");
} else {
return Optional.of(new ConsumerGroupMetadata(
groupId,
JoinGroupRequest.UNKNOWN_GENERATION_ID,
JoinGroupRequest.UNKNOWN_MEMBER_ID,
groupInstanceId
));
}
}
return Optional.empty();
}
/**
* poll implementation using {@link ApplicationEventHandler}.
* 1. Poll for background events. If there's a fetch response event, process the record and return it. If it is
* another type of event, process it.
* 2. Send fetches if needed.
* If the timeout expires, return an empty ConsumerRecord.
*
* @param timeout timeout of the poll loop
* @return ConsumerRecord. It can be empty if time timeout expires.
*
* @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
* function is called
* @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
* this function is called
* @throws org.apache.kafka.common.errors.RecordTooLargeException if the fetched record is larger than the maximum
* allowable size
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
* @throws java.lang.IllegalStateException if the consumer is not subscribed to any topics or manually assigned any
* partitions to consume from or an unexpected error occurred
* @throws org.apache.kafka.clients.consumer.OffsetOutOfRangeException if the fetch position of the consumer is
* out of range and no offset reset policy is configured.
* @throws org.apache.kafka.common.errors.TopicAuthorizationException if the consumer is not authorized to read
* from a partition
* @throws org.apache.kafka.common.errors.SerializationException if the fetched records cannot be deserialized
* @throws org.apache.kafka.common.errors.UnsupportedAssignorException if the `group.remote.assignor` configuration
* is set to an assignor that is not available on the broker.
*/
@Override
public ConsumerRecords<K, V> poll(final Duration timeout) {
Timer timer = time.timer(timeout);
acquireAndEnsureOpen();
try {
kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
if (subscriptions.hasNoSubscriptionOrUserAssignment()) {
throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions");
}
do {
// Make sure to let the background thread know that we are still polling.
applicationEventHandler.add(new PollApplicationEvent(timer.currentTimeMs()));
// We must not allow wake-ups between polling for fetches and returning the records.
// If the polled fetches are not empty the consumed position has already been updated in the polling
// of the fetches. A wakeup between returned fetches and returning records would lead to never
// returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches.
wakeupTrigger.maybeTriggerWakeup();
updateAssignmentMetadataIfNeeded(timer);
if (isGenerationKnownOrPartitionsUserAssigned()) {
final Fetch<K, V> fetch = pollForFetches(timer);
if (!fetch.isEmpty()) {
if (fetch.records().isEmpty()) {
log.trace("Returning empty records from `poll()` "
+ "since the consumer's position has advanced for at least one topic partition");
}
return interceptors.onConsume(new ConsumerRecords<>(fetch.records()));
}
} else {
timer.update();
}
// We will wait for retryBackoffMs
} while (timer.notExpired());
return ConsumerRecords.empty();
} finally {
kafkaConsumerMetrics.recordPollEnd(timer.currentTimeMs());
release();
}
}
private boolean isGenerationKnownOrPartitionsUserAssigned() {
if (subscriptions.hasAutoAssignedPartitions()) {
return groupMetadata.filter(g -> g.generationId() != JoinGroupRequest.UNKNOWN_GENERATION_ID).isPresent();
}
return true;
}
/**
* Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and
* partitions.
*/
@Override
public void commitSync() {
commitSync(Duration.ofMillis(defaultApiTimeoutMs));
}
/**
* This method sends a commit event to the EventHandler and return.
*/
@Override
public void commitAsync() {
commitAsync(null);
}
@Override
public void commitAsync(OffsetCommitCallback callback) {
commitAsync(subscriptions.allConsumed(), callback);
}
@Override
public void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback) {
acquireAndEnsureOpen();
try {
AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets);
CompletableFuture<Void> future = commit(asyncCommitEvent);
future.whenComplete((r, t) -> {
if (t == null) {
offsetCommitCallbackInvoker.enqueueInterceptorInvocation(offsets);
}
if (callback == null) {
if (t != null) {
log.error("Offset commit with offsets {} failed", offsets, t);
}
return;
}
offsetCommitCallbackInvoker.enqueueUserCallbackInvocation(callback, offsets, (Exception) t);
});
} finally {
release();
}
}
private CompletableFuture<Void> commit(final CommitApplicationEvent commitEvent) {
maybeInvokeCommitCallbacks();
maybeThrowFencedInstanceException();
maybeThrowInvalidGroupIdException();
Map<TopicPartition, OffsetAndMetadata> offsets = commitEvent.offsets();
log.debug("Committing offsets: {}", offsets);
offsets.forEach(this::updateLastSeenEpochIfNewer);
if (offsets.isEmpty()) {
return CompletableFuture.completedFuture(null);
}
applicationEventHandler.add(commitEvent);
return commitEvent.future();
}
@Override
public void seek(TopicPartition partition, long offset) {
if (offset < 0)
throw new IllegalArgumentException("seek offset must not be a negative number");
acquireAndEnsureOpen();
try {
log.info("Seeking to offset {} for partition {}", offset, partition);
SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition(
offset,
Optional.empty(), // This will ensure we skip validation
metadata.currentLeader(partition));
subscriptions.seekUnvalidated(partition, newPosition);
} finally {
release();
}
}
@Override
public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) {
long offset = offsetAndMetadata.offset();
if (offset < 0) {
throw new IllegalArgumentException("seek offset must not be a negative number");
}
acquireAndEnsureOpen();
try {
if (offsetAndMetadata.leaderEpoch().isPresent()) {
log.info("Seeking to offset {} for partition {} with epoch {}",
offset, partition, offsetAndMetadata.leaderEpoch().get());
} else {
log.info("Seeking to offset {} for partition {}", offset, partition);
}
Metadata.LeaderAndEpoch currentLeaderAndEpoch = metadata.currentLeader(partition);
SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition(
offsetAndMetadata.offset(),
offsetAndMetadata.leaderEpoch(),
currentLeaderAndEpoch);
updateLastSeenEpochIfNewer(partition, offsetAndMetadata);
subscriptions.seekUnvalidated(partition, newPosition);
} finally {
release();
}
}
@Override
public void seekToBeginning(Collection<TopicPartition> partitions) {
if (partitions == null)
throw new IllegalArgumentException("Partitions collection cannot be null");
acquireAndEnsureOpen();
try {
Collection<TopicPartition> parts = partitions.isEmpty() ? subscriptions.assignedPartitions() : partitions;
subscriptions.requestOffsetReset(parts, OffsetResetStrategy.EARLIEST);
} finally {
release();
}
}
@Override
public void seekToEnd(Collection<TopicPartition> partitions) {
if (partitions == null)
throw new IllegalArgumentException("Partitions collection cannot be null");
acquireAndEnsureOpen();
try {
Collection<TopicPartition> parts = partitions.isEmpty() ? subscriptions.assignedPartitions() : partitions;
subscriptions.requestOffsetReset(parts, OffsetResetStrategy.LATEST);
} finally {
release();
}
}
@Override
public long position(TopicPartition partition) {
return position(partition, Duration.ofMillis(defaultApiTimeoutMs));
}
@Override
public long position(TopicPartition partition, Duration timeout) {
acquireAndEnsureOpen();
try {
if (!subscriptions.isAssigned(partition))
throw new IllegalStateException("You can only check the position for partitions assigned to this consumer.");
Timer timer = time.timer(timeout);
do {
SubscriptionState.FetchPosition position = subscriptions.validPosition(partition);
if (position != null)
return position.offset;
updateFetchPositions(timer);
} while (timer.notExpired());
throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the position " +
"for partition " + partition + " could be determined");
} finally {
release();
}
}
@Override
@Deprecated
public OffsetAndMetadata committed(TopicPartition partition) {
return committed(partition, Duration.ofMillis(defaultApiTimeoutMs));
}
@Override
@Deprecated
public OffsetAndMetadata committed(TopicPartition partition, Duration timeout) {
return committed(Collections.singleton(partition), timeout).get(partition);
}
@Override
public Map<TopicPartition, OffsetAndMetadata> committed(final Set<TopicPartition> partitions) {
return committed(partitions, Duration.ofMillis(defaultApiTimeoutMs));
}
@Override
public Map<TopicPartition, OffsetAndMetadata> committed(final Set<TopicPartition> partitions,
final Duration timeout) {
acquireAndEnsureOpen();
long start = time.nanoseconds();
try {
maybeThrowInvalidGroupIdException();
if (partitions.isEmpty()) {
return Collections.emptyMap();
}
final FetchCommittedOffsetsApplicationEvent event = new FetchCommittedOffsetsApplicationEvent(
partitions,
timeout.toMillis());
wakeupTrigger.setActiveTask(event.future());
try {
final Map<TopicPartition, OffsetAndMetadata> committedOffsets = applicationEventHandler.addAndGet(event,
time.timer(timeout));
committedOffsets.forEach(this::updateLastSeenEpochIfNewer);
return committedOffsets;
} catch (TimeoutException e) {
throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the last " +
"committed offset for partitions " + partitions + " could be determined. Try tuning " +
ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG + " larger to relax the threshold.");
} finally {
wakeupTrigger.clearTask();
}
} finally {
kafkaConsumerMetrics.recordCommitted(time.nanoseconds() - start);
release();
}
}
private void maybeThrowInvalidGroupIdException() {
if (!groupMetadata.isPresent()) {
throw new InvalidGroupIdException("To use the group management or offset commit APIs, you must " +
"provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.");
}
}
@Override
public Map<MetricName, ? extends Metric> metrics() {
return Collections.unmodifiableMap(metrics.metrics());
}
@Override
public List<PartitionInfo> partitionsFor(String topic) {
return partitionsFor(topic, Duration.ofMillis(defaultApiTimeoutMs));
}
@Override
public List<PartitionInfo> partitionsFor(String topic, Duration timeout) {
acquireAndEnsureOpen();
try {
Cluster cluster = this.metadata.fetch();
List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
if (!parts.isEmpty())
return parts;
if (timeout.toMillis() == 0L) {
throw new TimeoutException();
}
final TopicMetadataApplicationEvent topicMetadataApplicationEvent =
new TopicMetadataApplicationEvent(topic, timeout.toMillis());
wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future());
try {
Map<String, List<PartitionInfo>> topicMetadata =
applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout));
return topicMetadata.getOrDefault(topic, Collections.emptyList());
} finally {
wakeupTrigger.clearTask();