/
ConsumerCoordinator.java
1638 lines (1425 loc) · 81.1 KB
/
ConsumerCoordinator.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.GroupRebalanceConfig;
import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator;
import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.RebalanceInProgressException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.errors.UnstableOffsetCommitException;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.metrics.Measurable;
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.Max;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter;
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 java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static org.apache.kafka.clients.consumer.ConsumerConfig.ASSIGN_FROM_SUBSCRIBED_ASSIGNORS;
import static org.apache.kafka.clients.consumer.CooperativeStickyAssignor.COOPERATIVE_STICKY_ASSIGNOR_NAME;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.refreshCommittedOffsets;
/**
* This class manages the coordination process with the consumer coordinator.
*/
public final class ConsumerCoordinator extends AbstractCoordinator {
private final static TopicPartitionComparator COMPARATOR = new TopicPartitionComparator();
private final GroupRebalanceConfig rebalanceConfig;
private final Logger log;
private final List<ConsumerPartitionAssignor> assignors;
private final ConsumerMetadata metadata;
private final ConsumerCoordinatorMetrics coordinatorMetrics;
private final SubscriptionState subscriptions;
private final OffsetCommitCallback defaultOffsetCommitCallback;
private final boolean autoCommitEnabled;
private final int autoCommitIntervalMs;
private final ConsumerInterceptors<?, ?> interceptors;
// track number of async commits for which callback must be called
// package private for testing
final AtomicInteger inFlightAsyncCommits;
// track the number of pending async commits waiting on the coordinator lookup to complete
private final AtomicInteger pendingAsyncCommits;
// this collection must be thread-safe because it is modified from the response handler
// of offset commit requests, which may be invoked from the heartbeat thread
private final ConcurrentLinkedQueue<OffsetCommitCompletion> completedOffsetCommits;
private final AtomicBoolean asyncCommitFenced;
private final boolean throwOnFetchStableOffsetsUnsupported;
private final Optional<String> rackId;
private boolean isLeader = false;
private Set<String> joinedSubscription;
private MetadataSnapshot metadataSnapshot;
private MetadataSnapshot assignmentSnapshot;
private Timer nextAutoCommitTimer;
private ConsumerGroupMetadata groupMetadata;
// hold onto request&future for committed offset requests to enable async calls.
private PendingCommittedOffsetRequest pendingCommittedOffsetRequest = null;
private static class PendingCommittedOffsetRequest {
private final Set<TopicPartition> requestedPartitions;
private final Generation requestedGeneration;
private final RequestFuture<Map<TopicPartition, OffsetAndMetadata>> response;
private PendingCommittedOffsetRequest(final Set<TopicPartition> requestedPartitions,
final Generation generationAtRequestTime,
final RequestFuture<Map<TopicPartition, OffsetAndMetadata>> response) {
this.requestedPartitions = Objects.requireNonNull(requestedPartitions);
this.response = Objects.requireNonNull(response);
this.requestedGeneration = generationAtRequestTime;
}
private boolean sameRequest(final Set<TopicPartition> currentRequest, final Generation currentGeneration) {
return Objects.equals(requestedGeneration, currentGeneration) && requestedPartitions.equals(currentRequest);
}
}
private final RebalanceProtocol protocol;
// Wraps the logic for invoking the ConsumerRebalanceListener methods
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
// pending commit offset request in onJoinPrepare
private RequestFuture<Void> autoCommitOffsetRequestFuture = null;
// a timer for join prepare to know when to stop.
// it'll set to rebalance timeout so that the member can join the group successfully
// even though offset commit failed.
private Timer joinPrepareTimer = null;
/**
* Initialize the coordination manager.
*/
public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig,
LogContext logContext,
ConsumerNetworkClient client,
List<ConsumerPartitionAssignor> assignors,
ConsumerMetadata metadata,
SubscriptionState subscriptions,
Metrics metrics,
String metricGrpPrefix,
Time time,
boolean autoCommitEnabled,
int autoCommitIntervalMs,
ConsumerInterceptors<?, ?> interceptors,
boolean throwOnFetchStableOffsetsUnsupported,
String rackId,
Optional<ClientTelemetryReporter> clientTelemetryReporter) {
super(rebalanceConfig,
logContext,
client,
metrics,
metricGrpPrefix,
time,
clientTelemetryReporter);
this.rebalanceConfig = rebalanceConfig;
this.log = logContext.logger(ConsumerCoordinator.class);
this.metadata = metadata;
this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : Optional.of(rackId);
this.metadataSnapshot = new MetadataSnapshot(this.rackId, subscriptions, metadata.fetch(), metadata.updateVersion());
this.subscriptions = subscriptions;
this.defaultOffsetCommitCallback = new DefaultOffsetCommitCallback();
this.autoCommitEnabled = autoCommitEnabled;
this.autoCommitIntervalMs = autoCommitIntervalMs;
this.assignors = assignors;
this.completedOffsetCommits = new ConcurrentLinkedQueue<>();
this.coordinatorMetrics = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix);
this.interceptors = interceptors;
this.inFlightAsyncCommits = new AtomicInteger();
this.pendingAsyncCommits = new AtomicInteger();
this.asyncCommitFenced = new AtomicBoolean(false);
this.groupMetadata = new ConsumerGroupMetadata(rebalanceConfig.groupId,
JoinGroupRequest.UNKNOWN_GENERATION_ID, JoinGroupRequest.UNKNOWN_MEMBER_ID, rebalanceConfig.groupInstanceId);
this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
if (autoCommitEnabled)
this.nextAutoCommitTimer = time.timer(autoCommitIntervalMs);
// select the rebalance protocol such that:
// 1. only consider protocols that are supported by all the assignors. If there is no common protocols supported
// across all the assignors, throw an exception.
// 2. if there are multiple protocols that are commonly supported, select the one with the highest id (i.e. the
// id number indicates how advanced the protocol is).
// we know there are at least one assignor in the list, no need to double check for NPE
if (!assignors.isEmpty()) {
List<RebalanceProtocol> supportedProtocols = new ArrayList<>(assignors.get(0).supportedProtocols());
for (ConsumerPartitionAssignor assignor : assignors) {
supportedProtocols.retainAll(assignor.supportedProtocols());
}
if (supportedProtocols.isEmpty()) {
throw new IllegalArgumentException("Specified assignors " +
assignors.stream().map(ConsumerPartitionAssignor::name).collect(Collectors.toSet()) +
" do not have commonly supported rebalance protocol");
}
Collections.sort(supportedProtocols);
protocol = supportedProtocols.get(supportedProtocols.size() - 1);
} else {
protocol = null;
}
this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
time,
new RebalanceCallbackMetricsManager(metrics, metricGrpPrefix)
);
this.metadata.requestUpdate(true);
}
// package private for testing
boolean isLeader() {
return this.isLeader;
}
// package private for testing
SubscriptionState subscriptionState() {
return this.subscriptions;
}
@Override
public String protocolType() {
return ConsumerProtocol.PROTOCOL_TYPE;
}
@Override
protected JoinGroupRequestData.JoinGroupRequestProtocolCollection metadata() {
log.debug("Joining group with current subscription: {}", subscriptions.subscription());
this.joinedSubscription = subscriptions.subscription();
JoinGroupRequestData.JoinGroupRequestProtocolCollection protocolSet = new JoinGroupRequestData.JoinGroupRequestProtocolCollection();
List<String> topics = new ArrayList<>(joinedSubscription);
for (ConsumerPartitionAssignor assignor : assignors) {
Subscription subscription = new Subscription(topics,
assignor.subscriptionUserData(joinedSubscription),
subscriptions.assignedPartitionsList(),
generation().generationId,
rackId);
ByteBuffer metadata = ConsumerProtocol.serializeSubscription(subscription);
protocolSet.add(new JoinGroupRequestData.JoinGroupRequestProtocol()
.setName(assignor.name())
.setMetadata(Utils.toArray(metadata)));
}
return protocolSet;
}
public void updatePatternSubscription(Cluster cluster) {
final Set<String> topicsToSubscribe = cluster.topics().stream()
.filter(subscriptions::matchesSubscribedPattern)
.collect(Collectors.toSet());
if (subscriptions.subscribeFromPattern(topicsToSubscribe))
metadata.requestUpdateForNewTopics();
}
private ConsumerPartitionAssignor lookupAssignor(String name) {
for (ConsumerPartitionAssignor assignor : this.assignors) {
if (assignor.name().equals(name))
return assignor;
}
return null;
}
private void maybeUpdateJoinedSubscription(Set<TopicPartition> assignedPartitions) {
if (subscriptions.hasPatternSubscription()) {
// Check if the assignment contains some topics that were not in the original
// subscription, if yes we will obey what leader has decided and add these topics
// into the subscriptions as long as they still match the subscribed pattern
Set<String> addedTopics = new HashSet<>();
// this is a copy because it's handed to listener below
for (TopicPartition tp : assignedPartitions) {
if (!joinedSubscription.contains(tp.topic()))
addedTopics.add(tp.topic());
}
if (!addedTopics.isEmpty()) {
Set<String> newSubscription = new HashSet<>(subscriptions.subscription());
Set<String> newJoinedSubscription = new HashSet<>(joinedSubscription);
newSubscription.addAll(addedTopics);
newJoinedSubscription.addAll(addedTopics);
if (this.subscriptions.subscribeFromPattern(newSubscription))
metadata.requestUpdateForNewTopics();
this.joinedSubscription = newJoinedSubscription;
}
}
}
private Exception invokeOnAssignment(final ConsumerPartitionAssignor assignor, final Assignment assignment) {
log.info("Notifying assignor about the new {}", assignment);
try {
assignor.onAssignment(assignment, groupMetadata);
} catch (Exception e) {
return e;
}
return null;
}
@Override
protected void onJoinComplete(int generation,
String memberId,
String assignmentStrategy,
ByteBuffer assignmentBuffer) {
log.debug("Executing onJoinComplete with generation {} and memberId {}", generation, memberId);
// Only the leader is responsible for monitoring for metadata changes (i.e. partition changes)
if (!isLeader)
assignmentSnapshot = null;
ConsumerPartitionAssignor assignor = lookupAssignor(assignmentStrategy);
if (assignor == null)
throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy);
// Give the assignor a chance to update internal state based on the received assignment
groupMetadata = new ConsumerGroupMetadata(rebalanceConfig.groupId, generation, memberId, rebalanceConfig.groupInstanceId);
SortedSet<TopicPartition> ownedPartitions = new TreeSet<>(COMPARATOR);
ownedPartitions.addAll(subscriptions.assignedPartitions());
// should at least encode the short version
if (assignmentBuffer.remaining() < 2)
throw new IllegalStateException("There are insufficient bytes available to read assignment from the sync-group response (" +
"actual byte size " + assignmentBuffer.remaining() + ") , this is not expected; " +
"it is possible that the leader's assign function is buggy and did not return any assignment for this member, " +
"or because static member is configured and the protocol is buggy hence did not get the assignment for this member");
Assignment assignment = ConsumerProtocol.deserializeAssignment(assignmentBuffer);
SortedSet<TopicPartition> assignedPartitions = new TreeSet<>(COMPARATOR);
assignedPartitions.addAll(assignment.partitions());
if (!subscriptions.checkAssignmentMatchedSubscription(assignedPartitions)) {
final String fullReason = String.format("received assignment %s does not match the current subscription %s; " +
"it is likely that the subscription has changed since we joined the group, will re-join with current subscription",
assignment.partitions(), subscriptions.prettyString());
requestRejoin("received assignment does not match the current subscription", fullReason);
return;
}
final AtomicReference<Exception> firstException = new AtomicReference<>(null);
SortedSet<TopicPartition> addedPartitions = new TreeSet<>(COMPARATOR);
addedPartitions.addAll(assignedPartitions);
addedPartitions.removeAll(ownedPartitions);
if (protocol == RebalanceProtocol.COOPERATIVE) {
SortedSet<TopicPartition> revokedPartitions = new TreeSet<>(COMPARATOR);
revokedPartitions.addAll(ownedPartitions);
revokedPartitions.removeAll(assignedPartitions);
log.info("Updating assignment with\n" +
"\tAssigned partitions: {}\n" +
"\tCurrent owned partitions: {}\n" +
"\tAdded partitions (assigned - owned): {}\n" +
"\tRevoked partitions (owned - assigned): {}\n",
assignedPartitions,
ownedPartitions,
addedPartitions,
revokedPartitions
);
if (!revokedPartitions.isEmpty()) {
// Revoke partitions that were previously owned but no longer assigned;
// note that we should only change the assignment (or update the assignor's state)
// AFTER we've triggered the revoke callback
firstException.compareAndSet(null, rebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions));
// If revoked any partitions, need to re-join the group afterwards
final String fullReason = String.format("need to revoke partitions %s as indicated " +
"by the current assignment and re-join", revokedPartitions);
requestRejoin("need to revoke partitions and re-join", fullReason);
}
}
// The leader may have assigned partitions which match our subscription pattern, but which
// were not explicitly requested, so we update the joined subscription here.
maybeUpdateJoinedSubscription(assignedPartitions);
// Catch any exception here to make sure we could complete the user callback.
firstException.compareAndSet(null, invokeOnAssignment(assignor, assignment));
// Reschedule the auto commit starting from now
if (autoCommitEnabled)
this.nextAutoCommitTimer.updateAndReset(autoCommitIntervalMs);
subscriptions.assignFromSubscribed(assignedPartitions);
// Add partitions that were not previously owned but are now assigned
firstException.compareAndSet(null, rebalanceListenerInvoker.invokePartitionsAssigned(addedPartitions));
if (firstException.get() != null) {
if (firstException.get() instanceof KafkaException) {
throw (KafkaException) firstException.get();
} else {
throw new KafkaException("User rebalance callback throws an error", firstException.get());
}
}
}
void maybeUpdateSubscriptionMetadata() {
int version = metadata.updateVersion();
if (version > metadataSnapshot.version) {
Cluster cluster = metadata.fetch();
if (subscriptions.hasPatternSubscription())
updatePatternSubscription(cluster);
// Update the current snapshot, which will be used to check for subscription
// changes that would require a rebalance (e.g. new partitions).
metadataSnapshot = new MetadataSnapshot(rackId, subscriptions, cluster, version);
}
}
private boolean coordinatorUnknownAndUnreadySync(Timer timer) {
return coordinatorUnknown() && !ensureCoordinatorReady(timer);
}
private boolean coordinatorUnknownAndUnreadyAsync() {
return coordinatorUnknown() && !ensureCoordinatorReadyAsync();
}
/**
* Poll for coordinator events. This ensures that the coordinator is known and that the consumer
* has joined the group (if it is using group management). This also handles periodic offset commits
* if they are enabled.
* <p>
* Returns early if the timeout expires or if waiting on rejoin is not required
*
* @param timer Timer bounding how long this method can block
* @param waitForJoinGroup Boolean flag indicating if we should wait until re-join group completes
* @throws KafkaException if the rebalance callback throws an exception
* @return true iff the operation succeeded
*/
public boolean poll(Timer timer, boolean waitForJoinGroup) {
maybeUpdateSubscriptionMetadata();
invokeCompletedOffsetCommitCallbacks();
if (subscriptions.hasAutoAssignedPartitions()) {
if (protocol == null) {
throw new IllegalStateException("User configured " + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG +
" to empty while trying to subscribe for group protocol to auto assign partitions");
}
// Always update the heartbeat last poll time so that the heartbeat thread does not leave the
// group proactively due to application inactivity even if (say) the coordinator cannot be found.
pollHeartbeat(timer.currentTimeMs());
if (coordinatorUnknownAndUnreadySync(timer)) {
return false;
}
if (rejoinNeededOrPending()) {
// due to a race condition between the initial metadata fetch and the initial rebalance,
// we need to ensure that the metadata is fresh before joining initially. This ensures
// that we have matched the pattern against the cluster's topics at least once before joining.
if (subscriptions.hasPatternSubscription()) {
// For consumer group that uses pattern-based subscription, after a topic is created,
// any consumer that discovers the topic after metadata refresh can trigger rebalance
// across the entire consumer group. Multiple rebalances can be triggered after one topic
// creation if consumers refresh metadata at vastly different times. We can significantly
// reduce the number of rebalances caused by single topic creation by asking consumer to
// refresh metadata before re-joining the group as long as the refresh backoff time has
// passed.
if (this.metadata.timeToAllowUpdate(timer.currentTimeMs()) == 0) {
this.metadata.requestUpdate(true);
}
if (!client.ensureFreshMetadata(timer)) {
return false;
}
maybeUpdateSubscriptionMetadata();
}
// if not wait for join group, we would just use a timer of 0
if (!ensureActiveGroup(waitForJoinGroup ? timer : time.timer(0L))) {
// since we may use a different timer in the callee, we'd still need
// to update the original timer's current time after the call
timer.update(time.milliseconds());
return false;
}
}
} else {
// For manually assigned partitions, we do not try to pro-actively lookup coordinator;
// instead we only try to refresh metadata when necessary.
// If connections to all nodes fail, wakeups triggered while attempting to send fetch
// requests result in polls returning immediately, causing a tight loop of polls. Without
// the wakeup, poll() with no channels would block for the timeout, delaying re-connection.
// awaitMetadataUpdate() in ensureCoordinatorReady initiates new connections with configured backoff and avoids the busy loop.
if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
client.awaitMetadataUpdate(timer);
}
// if there is pending coordinator requests, ensure they have a chance to be transmitted.
client.pollNoWakeup();
}
maybeAutoCommitOffsetsAsync(timer.currentTimeMs());
return true;
}
/**
* Return the time to the next needed invocation of {@link ConsumerNetworkClient#poll(Timer)}.
* @param now current time in milliseconds
* @return the maximum time in milliseconds the caller should wait before the next invocation of poll()
*/
public long timeToNextPoll(long now) {
if (!autoCommitEnabled)
return timeToNextHeartbeat(now);
return Math.min(nextAutoCommitTimer.remainingMs(), timeToNextHeartbeat(now));
}
private void updateGroupSubscription(Set<String> topics) {
// the leader will begin watching for changes to any of the topics the group is interested in,
// which ensures that all metadata changes will eventually be seen
if (this.subscriptions.groupSubscribe(topics))
metadata.requestUpdateForNewTopics();
// update metadata (if needed) and keep track of the metadata used for assignment so that
// we can check after rebalance completion whether anything has changed
if (!client.ensureFreshMetadata(time.timer(Long.MAX_VALUE)))
throw new TimeoutException();
maybeUpdateSubscriptionMetadata();
}
private boolean isAssignFromSubscribedTopicsAssignor(String name) {
return ASSIGN_FROM_SUBSCRIBED_ASSIGNORS.contains(name);
}
/**
* user-customized assignor may have created some topics that are not in the subscription list
* and assign their partitions to the members; in this case we would like to update the leader's
* own metadata with the newly added topics so that it will not trigger a subsequent rebalance
* when these topics gets updated from metadata refresh.
*
* We skip the check for in-product assignors since this will not happen in in-product assignors.
*
* TODO: this is a hack and not something we want to support long-term unless we push regex into the protocol
* we may need to modify the ConsumerPartitionAssignor API to better support this case.
*
* @param assignorName the selected assignor name
* @param assignments the assignments after assignor assigned
* @param allSubscribedTopics all consumers' subscribed topics
*/
private void maybeUpdateGroupSubscription(String assignorName,
Map<String, Assignment> assignments,
Set<String> allSubscribedTopics) {
if (!isAssignFromSubscribedTopicsAssignor(assignorName)) {
Set<String> assignedTopics = new HashSet<>();
for (Assignment assigned : assignments.values()) {
for (TopicPartition tp : assigned.partitions())
assignedTopics.add(tp.topic());
}
if (!assignedTopics.containsAll(allSubscribedTopics)) {
SortedSet<String> notAssignedTopics = new TreeSet<>(allSubscribedTopics);
notAssignedTopics.removeAll(assignedTopics);
log.warn("The following subscribed topics are not assigned to any members: {} ", notAssignedTopics);
}
if (!allSubscribedTopics.containsAll(assignedTopics)) {
SortedSet<String> newlyAddedTopics = new TreeSet<>(assignedTopics);
newlyAddedTopics.removeAll(allSubscribedTopics);
log.info("The following not-subscribed topics are assigned, and their metadata will be " +
"fetched from the brokers: {}", newlyAddedTopics);
allSubscribedTopics.addAll(newlyAddedTopics);
updateGroupSubscription(allSubscribedTopics);
}
}
}
@Override
protected Map<String, ByteBuffer> onLeaderElected(String leaderId,
String assignmentStrategy,
List<JoinGroupResponseData.JoinGroupResponseMember> allSubscriptions,
boolean skipAssignment) {
ConsumerPartitionAssignor assignor = lookupAssignor(assignmentStrategy);
if (assignor == null)
throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy);
String assignorName = assignor.name();
Set<String> allSubscribedTopics = new HashSet<>();
Map<String, Subscription> subscriptions = new HashMap<>();
// collect all the owned partitions
Map<String, List<TopicPartition>> ownedPartitions = new HashMap<>();
for (JoinGroupResponseData.JoinGroupResponseMember memberSubscription : allSubscriptions) {
Subscription subscription = ConsumerProtocol.deserializeSubscription(ByteBuffer.wrap(memberSubscription.metadata()));
subscription.setGroupInstanceId(Optional.ofNullable(memberSubscription.groupInstanceId()));
subscriptions.put(memberSubscription.memberId(), subscription);
allSubscribedTopics.addAll(subscription.topics());
ownedPartitions.put(memberSubscription.memberId(), subscription.ownedPartitions());
}
// the leader will begin watching for changes to any of the topics the group is interested in,
// which ensures that all metadata changes will eventually be seen
updateGroupSubscription(allSubscribedTopics);
isLeader = true;
if (skipAssignment) {
log.info("Skipped assignment for returning static leader at generation {}. The static leader " +
"will continue with its existing assignment.", generation().generationId);
assignmentSnapshot = metadataSnapshot;
return Collections.emptyMap();
}
log.debug("Performing assignment using strategy {} with subscriptions {}", assignorName, subscriptions);
Map<String, Assignment> assignments = assignor.assign(metadata.fetch(), new GroupSubscription(subscriptions)).groupAssignment();
// skip the validation for built-in cooperative sticky assignor since we've considered
// the "generation" of ownedPartition inside the assignor
if (protocol == RebalanceProtocol.COOPERATIVE && !assignorName.equals(COOPERATIVE_STICKY_ASSIGNOR_NAME)) {
validateCooperativeAssignment(ownedPartitions, assignments);
}
maybeUpdateGroupSubscription(assignorName, assignments, allSubscribedTopics);
// metadataSnapshot could be updated when the subscription is updated therefore
// we must take the assignment snapshot after.
assignmentSnapshot = metadataSnapshot;
log.info("Finished assignment for group at generation {}: {}", generation().generationId, assignments);
Map<String, ByteBuffer> groupAssignment = new HashMap<>();
for (Map.Entry<String, Assignment> assignmentEntry : assignments.entrySet()) {
ByteBuffer buffer = ConsumerProtocol.serializeAssignment(assignmentEntry.getValue());
groupAssignment.put(assignmentEntry.getKey(), buffer);
}
return groupAssignment;
}
/**
* Used by COOPERATIVE rebalance protocol only.
*
* Validate the assignments returned by the assignor such that no owned partitions are going to
* be reassigned to a different consumer directly: if the assignor wants to reassign an owned partition,
* it must first remove it from the new assignment of the current owner so that it is not assigned to any
* member, and then in the next rebalance it can finally reassign those partitions not owned by anyone to consumers.
*/
private void validateCooperativeAssignment(final Map<String, List<TopicPartition>> ownedPartitions,
final Map<String, Assignment> assignments) {
Set<TopicPartition> totalRevokedPartitions = new HashSet<>();
SortedSet<TopicPartition> totalAddedPartitions = new TreeSet<>(COMPARATOR);
for (final Map.Entry<String, Assignment> entry : assignments.entrySet()) {
final Assignment assignment = entry.getValue();
final Set<TopicPartition> addedPartitions = new HashSet<>(assignment.partitions());
addedPartitions.removeAll(ownedPartitions.get(entry.getKey()));
final Set<TopicPartition> revokedPartitions = new HashSet<>(ownedPartitions.get(entry.getKey()));
revokedPartitions.removeAll(assignment.partitions());
totalAddedPartitions.addAll(addedPartitions);
totalRevokedPartitions.addAll(revokedPartitions);
}
// if there are overlap between revoked partitions and added partitions, it means some partitions
// immediately gets re-assigned to another member while it is still claimed by some member
totalAddedPartitions.retainAll(totalRevokedPartitions);
if (!totalAddedPartitions.isEmpty()) {
log.error("With the COOPERATIVE protocol, owned partitions cannot be " +
"reassigned to other members; however the assignor has reassigned partitions {} which are still owned " +
"by some members", totalAddedPartitions);
throw new IllegalStateException("Assignor supporting the COOPERATIVE protocol violates its requirements");
}
}
@Override
protected boolean onJoinPrepare(Timer timer, int generation, String memberId) {
log.debug("Executing onJoinPrepare with generation {} and memberId {}", generation, memberId);
if (joinPrepareTimer == null) {
// We should complete onJoinPrepare before rebalanceTimeoutMs,
// and continue to join group to avoid member got kicked out from group
joinPrepareTimer = time.timer(rebalanceConfig.rebalanceTimeoutMs);
} else {
joinPrepareTimer.update();
}
// async commit offsets prior to rebalance if auto-commit enabled
// and there is no in-flight offset commit request
if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
maybeMarkPartitionsPendingRevocation();
autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
}
// wait for commit offset response before timer expired
if (autoCommitOffsetRequestFuture != null) {
Timer pollTimer = timer.remainingMs() < joinPrepareTimer.remainingMs() ?
timer : joinPrepareTimer;
client.poll(autoCommitOffsetRequestFuture, pollTimer);
joinPrepareTimer.update();
// Keep retrying/waiting the offset commit when:
// 1. offset commit haven't done (and joinPrepareTimer not expired)
// 2. failed with retriable exception (and joinPrepareTimer not expired)
// Otherwise, continue to revoke partitions, ex:
// 1. if joinPrepareTimer has expired
// 2. if offset commit failed with non-retriable exception
// 3. if offset commit success
boolean onJoinPrepareAsyncCommitCompleted = true;
if (joinPrepareTimer.isExpired()) {
log.error("Asynchronous auto-commit of offsets failed: joinPrepare timeout. Will continue to join group");
} else if (!autoCommitOffsetRequestFuture.isDone()) {
onJoinPrepareAsyncCommitCompleted = false;
} else if (autoCommitOffsetRequestFuture.failed() && autoCommitOffsetRequestFuture.isRetriable()) {
log.debug("Asynchronous auto-commit of offsets failed with retryable error: {}. Will retry it.",
autoCommitOffsetRequestFuture.exception().getMessage());
onJoinPrepareAsyncCommitCompleted = false;
} else if (autoCommitOffsetRequestFuture.failed() && !autoCommitOffsetRequestFuture.isRetriable()) {
log.error("Asynchronous auto-commit of offsets failed: {}. Will continue to join group.",
autoCommitOffsetRequestFuture.exception().getMessage());
}
if (autoCommitOffsetRequestFuture.isDone()) {
autoCommitOffsetRequestFuture = null;
}
if (!onJoinPrepareAsyncCommitCompleted) {
pollTimer.sleep(Math.min(pollTimer.remainingMs(), rebalanceConfig.retryBackoffMs));
timer.update();
return false;
}
}
// the generation / member-id can possibly be reset by the heartbeat thread
// upon getting errors or heartbeat timeouts; in this case whatever is previously
// owned partitions would be lost, we should trigger the callback and cleanup the assignment;
// otherwise we can proceed normally and revoke the partitions depending on the protocol,
// and in that case we should only change the assignment AFTER the revoke callback is triggered
// so that users can still access the previously owned partitions to commit offsets etc.
Exception exception = null;
final SortedSet<TopicPartition> revokedPartitions = new TreeSet<>(COMPARATOR);
if (generation == Generation.NO_GENERATION.generationId ||
memberId.equals(Generation.NO_GENERATION.memberId)) {
revokedPartitions.addAll(subscriptions.assignedPartitions());
if (!revokedPartitions.isEmpty()) {
log.info("Giving away all assigned partitions as lost since generation/memberID has been reset," +
"indicating that consumer is in old state or no longer part of the group");
exception = rebalanceListenerInvoker.invokePartitionsLost(revokedPartitions);
subscriptions.assignFromSubscribed(Collections.emptySet());
}
} else {
switch (protocol) {
case EAGER:
// revoke all partitions
revokedPartitions.addAll(subscriptions.assignedPartitions());
exception = rebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions);
subscriptions.assignFromSubscribed(Collections.emptySet());
break;
case COOPERATIVE:
// only revoke those partitions that are not in the subscription anymore.
Set<TopicPartition> ownedPartitions = new HashSet<>(subscriptions.assignedPartitions());
revokedPartitions.addAll(ownedPartitions.stream()
.filter(tp -> !subscriptions.subscription().contains(tp.topic()))
.collect(Collectors.toSet()));
if (!revokedPartitions.isEmpty()) {
exception = rebalanceListenerInvoker.invokePartitionsRevoked(revokedPartitions);
ownedPartitions.removeAll(revokedPartitions);
subscriptions.assignFromSubscribed(ownedPartitions);
}
break;
}
}
isLeader = false;
subscriptions.resetGroupSubscription();
joinPrepareTimer = null;
autoCommitOffsetRequestFuture = null;
timer.update();
if (exception != null) {
throw new KafkaException("User rebalance callback throws an error", exception);
}
return true;
}
private void maybeMarkPartitionsPendingRevocation() {
if (protocol != RebalanceProtocol.EAGER) {
return;
}
// When asynchronously committing offsets prior to the revocation of a set of partitions, there will be a
// window of time between when the offset commit is sent and when it returns and revocation completes. It is
// possible for pending fetches for these partitions to return during this time, which means the application's
// position may get ahead of the committed position prior to revocation. This can cause duplicate consumption.
// To prevent this, we mark the partitions as "pending revocation," which stops the Fetcher from sending new
// fetches or returning data from previous fetches to the user.
Set<TopicPartition> partitions = subscriptions.assignedPartitions();
log.debug("Marking assigned partitions pending for revocation: {}", partitions);
subscriptions.markPendingRevocation(partitions);
}
@Override
public void onLeavePrepare() {
// Save the current Generation, as the hb thread can change it at any time
final Generation currentGeneration = generation();
log.debug("Executing onLeavePrepare with generation {}", currentGeneration);
// we should reset assignment and trigger the callback before leaving group
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(COMPARATOR);
droppedPartitions.addAll(subscriptions.assignedPartitions());
if (subscriptions.hasAutoAssignedPartitions() && !droppedPartitions.isEmpty()) {
final Exception e;
if ((currentGeneration.generationId == Generation.NO_GENERATION.generationId ||
currentGeneration.memberId.equals(Generation.NO_GENERATION.memberId)) ||
rebalanceInProgress()) {
e = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions);
} else {
e = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
}
subscriptions.assignFromSubscribed(Collections.emptySet());
if (e != null) {
throw new KafkaException("User rebalance callback throws an error", e);
}
}
}
/**
* @throws KafkaException if the callback throws exception
*/
@Override
public boolean rejoinNeededOrPending() {
if (!subscriptions.hasAutoAssignedPartitions())
return false;
// we need to rejoin if we performed the assignment and metadata has changed;
// also for those owned-but-no-longer-existed partitions we should drop them as lost
if (assignmentSnapshot != null && !assignmentSnapshot.matches(metadataSnapshot)) {
final String fullReason = String.format("cached metadata has changed from %s at the beginning of the rebalance to %s",
assignmentSnapshot, metadataSnapshot);
requestRejoinIfNecessary("cached metadata has changed", fullReason);
return true;
}
// we need to join if our subscription has changed since the last join
if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) {
final String fullReason = String.format("subscription has changed from %s at the beginning of the rebalance to %s",
joinedSubscription, subscriptions.subscription());
requestRejoinIfNecessary("subscription has changed", fullReason);
return true;
}
return super.rejoinNeededOrPending();
}
/**
* Refresh the committed offsets for partitions that require initialization.
*
* @param timer Timer bounding how long this method can block
* @return true iff the operation completed within the timeout
*/
public boolean initWithCommittedOffsetsIfNeeded(Timer timer) {
final Set<TopicPartition> initializingPartitions = subscriptions.initializingPartitions();
final Map<TopicPartition, OffsetAndMetadata> offsets = fetchCommittedOffsets(initializingPartitions, timer);
// "offsets" will be null if the offset fetch requests did not receive responses within the given timeout
if (offsets == null)
return false;
refreshCommittedOffsets(offsets, this.metadata, this.subscriptions);
return true;
}
/**
* Fetch the current committed offsets from the coordinator for a set of partitions.
*
* @param partitions The partitions to fetch offsets for
* @return A map from partition to the committed offset or null if the operation timed out
*/
public Map<TopicPartition, OffsetAndMetadata> fetchCommittedOffsets(final Set<TopicPartition> partitions,
final Timer timer) {
if (partitions.isEmpty()) return Collections.emptyMap();
final Generation generationForOffsetRequest = generationIfStable();
if (pendingCommittedOffsetRequest != null &&
!pendingCommittedOffsetRequest.sameRequest(partitions, generationForOffsetRequest)) {
// if we were waiting for a different request, then just clear it.
pendingCommittedOffsetRequest = null;
}
long attempts = 0L;
do {
if (!ensureCoordinatorReady(timer)) return null;
// contact coordinator to fetch committed offsets
final RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future;
if (pendingCommittedOffsetRequest != null) {
future = pendingCommittedOffsetRequest.response;
} else {
future = sendOffsetFetchRequest(partitions);
pendingCommittedOffsetRequest = new PendingCommittedOffsetRequest(partitions, generationForOffsetRequest, future);
}
client.poll(future, timer);
if (future.isDone()) {
pendingCommittedOffsetRequest = null;
if (future.succeeded()) {
return future.value();
} else if (!future.isRetriable()) {
throw future.exception();
} else {
timer.sleep(retryBackoff.backoff(attempts++));
}
} else {
return null;
}
} while (timer.notExpired());
return null;
}
/**
* Return the consumer group metadata.
*
* @return the current consumer group metadata
*/
public ConsumerGroupMetadata groupMetadata() {
return groupMetadata;
}
/**
* @throws KafkaException if the rebalance callback throws exception
*/
public void close(final Timer timer) {
// we do not need to re-enable wakeups since we are closing already
client.disableWakeups();
try {
maybeAutoCommitOffsetsSync(timer);
while (pendingAsyncCommits.get() > 0 && timer.notExpired()) {
ensureCoordinatorReady(timer);
client.poll(timer);
invokeCompletedOffsetCommitCallbacks();
}
} finally {
super.close(timer);
}
}
// visible for testing
void invokeCompletedOffsetCommitCallbacks() {
if (asyncCommitFenced.get()) {
throw new FencedInstanceIdException("Get fenced exception for group.instance.id "
+ rebalanceConfig.groupInstanceId.orElse("unset_instance_id")
+ ", current member.id is " + memberId());
}
while (true) {
OffsetCommitCompletion completion = completedOffsetCommits.poll();
if (completion == null) {
break;