/
Fetcher.java
1908 lines (1693 loc) · 96.2 KB
/
Fetcher.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.ClientResponse;
import org.apache.kafka.clients.FetchSessionHandler;
import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.clients.StaleMetadataException;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.LogTruncationException;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
import org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.metrics.Gauge;
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.metrics.stats.Meter;
import org.apache.kafka.common.metrics.stats.Min;
import org.apache.kafka.common.metrics.stats.Value;
import org.apache.kafka.common.metrics.stats.WindowedCount;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.record.ControlRecordType;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.ListOffsetsRequest;
import org.apache.kafka.common.requests.ListOffsetsResponse;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.CloseableIterator;
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.helpers.MessageFormatter;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.PriorityQueue;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
import static java.util.Collections.emptyList;
/**
* This class manages the fetching process with the brokers.
* <p>
* Thread-safety:
* Requests and responses of Fetcher may be processed by different threads since heartbeat
* thread may process responses. Other operations are single-threaded and invoked only from
* the thread polling the consumer.
* <ul>
* <li>If a response handler accesses any shared state of the Fetcher (e.g. FetchSessionHandler),
* all access to that state must be synchronized on the Fetcher instance.</li>
* <li>If a response handler accesses any shared state of the coordinator (e.g. SubscriptionState),
* it is assumed that all access to that state is synchronized on the coordinator instance by
* the caller.</li>
* <li>Responses that collate partial responses from multiple brokers (e.g. to list offsets) are
* synchronized on the response future.</li>
* <li>At most one request is pending for each node at any time. Nodes with pending requests are
* tracked and updated after processing the response. This ensures that any state (e.g. epoch)
* updated while processing responses on one thread are visible while creating the subsequent request
* on a different thread.</li>
* </ul>
*/
public class Fetcher<K, V> implements Closeable {
private final Logger log;
private final LogContext logContext;
private final ConsumerNetworkClient client;
private final Time time;
private final int minBytes;
private final int maxBytes;
private final int maxWaitMs;
private final int fetchSize;
private final long retryBackoffMs;
private final long requestTimeoutMs;
private final int maxPollRecords;
private final boolean checkCrcs;
private final String clientRackId;
private final ConsumerMetadata metadata;
private final FetchManagerMetrics sensors;
private final SubscriptionState subscriptions;
private final ConcurrentLinkedQueue<CompletedFetch> completedFetches;
private final BufferSupplier decompressionBufferSupplier = BufferSupplier.create();
private final Deserializer<K> keyDeserializer;
private final Deserializer<V> valueDeserializer;
private final IsolationLevel isolationLevel;
private final Map<Integer, FetchSessionHandler> sessionHandlers;
private final AtomicReference<RuntimeException> cachedListOffsetsException = new AtomicReference<>();
private final AtomicReference<RuntimeException> cachedOffsetForLeaderException = new AtomicReference<>();
private final OffsetsForLeaderEpochClient offsetsForLeaderEpochClient;
private final Set<Integer> nodesWithPendingFetchRequests;
private final ApiVersions apiVersions;
private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1);
private CompletedFetch nextInLineFetch = null;
public Fetcher(LogContext logContext,
ConsumerNetworkClient client,
int minBytes,
int maxBytes,
int maxWaitMs,
int fetchSize,
int maxPollRecords,
boolean checkCrcs,
String clientRackId,
Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer,
ConsumerMetadata metadata,
SubscriptionState subscriptions,
Metrics metrics,
FetcherMetricsRegistry metricsRegistry,
Time time,
long retryBackoffMs,
long requestTimeoutMs,
IsolationLevel isolationLevel,
ApiVersions apiVersions) {
this.log = logContext.logger(Fetcher.class);
this.logContext = logContext;
this.time = time;
this.client = client;
this.metadata = metadata;
this.subscriptions = subscriptions;
this.minBytes = minBytes;
this.maxBytes = maxBytes;
this.maxWaitMs = maxWaitMs;
this.fetchSize = fetchSize;
this.maxPollRecords = maxPollRecords;
this.checkCrcs = checkCrcs;
this.clientRackId = clientRackId;
this.keyDeserializer = keyDeserializer;
this.valueDeserializer = valueDeserializer;
this.completedFetches = new ConcurrentLinkedQueue<>();
this.sensors = new FetchManagerMetrics(metrics, metricsRegistry);
this.retryBackoffMs = retryBackoffMs;
this.requestTimeoutMs = requestTimeoutMs;
this.isolationLevel = isolationLevel;
this.apiVersions = apiVersions;
this.sessionHandlers = new HashMap<>();
this.offsetsForLeaderEpochClient = new OffsetsForLeaderEpochClient(client, logContext);
this.nodesWithPendingFetchRequests = new HashSet<>();
}
/**
* Represents data about an offset returned by a broker.
*/
static class ListOffsetData {
final long offset;
final Long timestamp; // null if the broker does not support returning timestamps
final Optional<Integer> leaderEpoch; // empty if the leader epoch is not known
ListOffsetData(long offset, Long timestamp, Optional<Integer> leaderEpoch) {
this.offset = offset;
this.timestamp = timestamp;
this.leaderEpoch = leaderEpoch;
}
}
/**
* Return whether we have any completed fetches pending return to the user. This method is thread-safe. Has
* visibility for testing.
* @return true if there are completed fetches, false otherwise
*/
protected boolean hasCompletedFetches() {
return !completedFetches.isEmpty();
}
/**
* Return whether we have any completed fetches that are fetchable. This method is thread-safe.
* @return true if there are completed fetches that can be returned, false otherwise
*/
public boolean hasAvailableFetches() {
return completedFetches.stream().anyMatch(fetch -> subscriptions.isFetchable(fetch.partition));
}
/**
* Set-up a fetch request for any node that we have assigned partitions for which doesn't already have
* an in-flight fetch or pending fetch data.
* @return number of fetches sent
*/
public synchronized int sendFetches() {
// Update metrics in case there was an assignment change
sensors.maybeUpdateAssignment(subscriptions);
Map<Node, FetchSessionHandler.FetchRequestData> fetchRequestMap = prepareFetchRequests();
for (Map.Entry<Node, FetchSessionHandler.FetchRequestData> entry : fetchRequestMap.entrySet()) {
final Node fetchTarget = entry.getKey();
final FetchSessionHandler.FetchRequestData data = entry.getValue();
final FetchRequest.Builder request = FetchRequest.Builder
.forConsumer(this.maxWaitMs, this.minBytes, data.toSend())
.isolationLevel(isolationLevel)
.setMaxBytes(this.maxBytes)
.metadata(data.metadata())
.toForget(data.toForget())
.rackId(clientRackId);
if (log.isDebugEnabled()) {
log.debug("Sending {} {} to broker {}", isolationLevel, data.toString(), fetchTarget);
}
RequestFuture<ClientResponse> future = client.send(fetchTarget, request);
// We add the node to the set of nodes with pending fetch requests before adding the
// listener because the future may have been fulfilled on another thread (e.g. during a
// disconnection being handled by the heartbeat thread) which will mean the listener
// will be invoked synchronously.
this.nodesWithPendingFetchRequests.add(entry.getKey().id());
future.addListener(new RequestFutureListener<ClientResponse>() {
@Override
public void onSuccess(ClientResponse resp) {
synchronized (Fetcher.this) {
try {
@SuppressWarnings("unchecked")
FetchResponse response = (FetchResponse) resp.responseBody();
FetchSessionHandler handler = sessionHandler(fetchTarget.id());
if (handler == null) {
log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.",
fetchTarget.id());
return;
}
if (!handler.handleResponse(response)) {
return;
}
Set<TopicPartition> partitions = new HashSet<>(response.responseData().keySet());
FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions);
for (Map.Entry<TopicPartition, FetchResponseData.PartitionData> entry : response.responseData().entrySet()) {
TopicPartition partition = entry.getKey();
FetchRequest.PartitionData requestData = data.sessionPartitions().get(partition);
if (requestData == null) {
String message;
if (data.metadata().isFull()) {
message = MessageFormatter.arrayFormat(
"Response for missing full request partition: partition={}; metadata={}",
new Object[]{partition, data.metadata()}).getMessage();
} else {
message = MessageFormatter.arrayFormat(
"Response for missing session request partition: partition={}; metadata={}; toSend={}; toForget={}",
new Object[]{partition, data.metadata(), data.toSend(), data.toForget()}).getMessage();
}
// Received fetch response for missing session partition
throw new IllegalStateException(message);
} else {
long fetchOffset = requestData.fetchOffset;
FetchResponseData.PartitionData partitionData = entry.getValue();
log.debug("Fetch {} at offset {} for partition {} returned fetch data {}",
isolationLevel, fetchOffset, partition, partitionData);
Iterator<? extends RecordBatch> batches = FetchResponse.recordsOrFail(partitionData).batches().iterator();
short responseVersion = resp.requestHeader().apiVersion();
completedFetches.add(new CompletedFetch(partition, partitionData,
metricAggregator, batches, fetchOffset, responseVersion));
}
}
sensors.fetchLatency.record(resp.requestLatencyMs());
} finally {
nodesWithPendingFetchRequests.remove(fetchTarget.id());
}
}
}
@Override
public void onFailure(RuntimeException e) {
synchronized (Fetcher.this) {
try {
FetchSessionHandler handler = sessionHandler(fetchTarget.id());
if (handler != null) {
handler.handleError(e);
}
} finally {
nodesWithPendingFetchRequests.remove(fetchTarget.id());
}
}
}
});
}
return fetchRequestMap.size();
}
/**
* Get topic metadata for all topics in the cluster
* @param timer Timer bounding how long this method can block
* @return The map of topics with their partition information
*/
public Map<String, List<PartitionInfo>> getAllTopicMetadata(Timer timer) {
return getTopicMetadata(MetadataRequest.Builder.allTopics(), timer);
}
/**
* Get metadata for all topics present in Kafka cluster
*
* @param request The MetadataRequest to send
* @param timer Timer bounding how long this method can block
* @return The map of topics with their partition information
*/
public Map<String, List<PartitionInfo>> getTopicMetadata(MetadataRequest.Builder request, Timer timer) {
// Save the round trip if no topics are requested.
if (!request.isAllTopics() && request.emptyTopicList())
return Collections.emptyMap();
do {
RequestFuture<ClientResponse> future = sendMetadataRequest(request);
client.poll(future, timer);
if (future.failed() && !future.isRetriable())
throw future.exception();
if (future.succeeded()) {
MetadataResponse response = (MetadataResponse) future.value().responseBody();
Cluster cluster = response.cluster();
Set<String> unauthorizedTopics = cluster.unauthorizedTopics();
if (!unauthorizedTopics.isEmpty())
throw new TopicAuthorizationException(unauthorizedTopics);
boolean shouldRetry = false;
Map<String, Errors> errors = response.errors();
if (!errors.isEmpty()) {
// if there were errors, we need to check whether they were fatal or whether
// we should just retry
log.debug("Topic metadata fetch included errors: {}", errors);
for (Map.Entry<String, Errors> errorEntry : errors.entrySet()) {
String topic = errorEntry.getKey();
Errors error = errorEntry.getValue();
if (error == Errors.INVALID_TOPIC_EXCEPTION)
throw new InvalidTopicException("Topic '" + topic + "' is invalid");
else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION)
// if a requested topic is unknown, we just continue and let it be absent
// in the returned map
continue;
else if (error.exception() instanceof RetriableException)
shouldRetry = true;
else
throw new KafkaException("Unexpected error fetching metadata for topic " + topic,
error.exception());
}
}
if (!shouldRetry) {
HashMap<String, List<PartitionInfo>> topicsPartitionInfos = new HashMap<>();
for (String topic : cluster.topics())
topicsPartitionInfos.put(topic, cluster.partitionsForTopic(topic));
return topicsPartitionInfos;
}
}
timer.sleep(retryBackoffMs);
} while (timer.notExpired());
throw new TimeoutException("Timeout expired while fetching topic metadata");
}
/**
* Send Metadata Request to least loaded node in Kafka cluster asynchronously
* @return A future that indicates result of sent metadata request
*/
private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builder request) {
final Node node = client.leastLoadedNode();
if (node == null)
return RequestFuture.noBrokersAvailable();
else
return client.send(node, request);
}
private Long offsetResetStrategyTimestamp(final TopicPartition partition) {
OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
if (strategy == OffsetResetStrategy.EARLIEST)
return ListOffsetsRequest.EARLIEST_TIMESTAMP;
else if (strategy == OffsetResetStrategy.LATEST)
return ListOffsetsRequest.LATEST_TIMESTAMP;
else
return null;
}
private OffsetResetStrategy timestampToOffsetResetStrategy(long timestamp) {
if (timestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP)
return OffsetResetStrategy.EARLIEST;
else if (timestamp == ListOffsetsRequest.LATEST_TIMESTAMP)
return OffsetResetStrategy.LATEST;
else
return null;
}
/**
* Reset offsets for all assigned partitions that require it.
*
* @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
* and one or more partitions aren't awaiting a seekToBeginning() or seekToEnd().
*/
public void resetOffsetsIfNeeded() {
// Raise exception from previous offset fetch if there is one
RuntimeException exception = cachedListOffsetsException.getAndSet(null);
if (exception != null)
throw exception;
Set<TopicPartition> partitions = subscriptions.partitionsNeedingReset(time.milliseconds());
if (partitions.isEmpty())
return;
final Map<TopicPartition, Long> offsetResetTimestamps = new HashMap<>();
for (final TopicPartition partition : partitions) {
Long timestamp = offsetResetStrategyTimestamp(partition);
if (timestamp != null)
offsetResetTimestamps.put(partition, timestamp);
}
resetOffsetsAsync(offsetResetTimestamps);
}
/**
* Validate offsets for all assigned partitions for which a leader change has been detected.
*/
public void validateOffsetsIfNeeded() {
RuntimeException exception = cachedOffsetForLeaderException.getAndSet(null);
if (exception != null)
throw exception;
// Validate each partition against the current leader and epoch
// If we see a new metadata version, check all partitions
validatePositionsOnMetadataChange();
// Collect positions needing validation, with backoff
Map<TopicPartition, FetchPosition> partitionsToValidate = subscriptions
.partitionsNeedingValidation(time.milliseconds())
.stream()
.filter(tp -> subscriptions.position(tp) != null)
.collect(Collectors.toMap(Function.identity(), subscriptions::position));
validateOffsetsAsync(partitionsToValidate);
}
public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartition, Long> timestampsToSearch,
Timer timer) {
metadata.addTransientTopics(topicsForPartitions(timestampsToSearch.keySet()));
try {
Map<TopicPartition, ListOffsetData> fetchedOffsets = fetchOffsetsByTimes(timestampsToSearch,
timer, true).fetchedOffsets;
HashMap<TopicPartition, OffsetAndTimestamp> offsetsByTimes = new HashMap<>(timestampsToSearch.size());
for (Map.Entry<TopicPartition, Long> entry : timestampsToSearch.entrySet())
offsetsByTimes.put(entry.getKey(), null);
for (Map.Entry<TopicPartition, ListOffsetData> entry : fetchedOffsets.entrySet()) {
// 'entry.getValue().timestamp' will not be null since we are guaranteed
// to work with a v1 (or later) ListOffset request
ListOffsetData offsetData = entry.getValue();
offsetsByTimes.put(entry.getKey(), new OffsetAndTimestamp(offsetData.offset, offsetData.timestamp,
offsetData.leaderEpoch));
}
return offsetsByTimes;
} finally {
metadata.clearTransientTopics();
}
}
private ListOffsetResult fetchOffsetsByTimes(Map<TopicPartition, Long> timestampsToSearch,
Timer timer,
boolean requireTimestamps) {
ListOffsetResult result = new ListOffsetResult();
if (timestampsToSearch.isEmpty())
return result;
Map<TopicPartition, Long> remainingToSearch = new HashMap<>(timestampsToSearch);
do {
RequestFuture<ListOffsetResult> future = sendListOffsetsRequests(remainingToSearch, requireTimestamps);
client.poll(future, timer);
if (!future.isDone()) {
break;
} else if (future.succeeded()) {
ListOffsetResult value = future.value();
result.fetchedOffsets.putAll(value.fetchedOffsets);
remainingToSearch.keySet().retainAll(value.partitionsToRetry);
} else if (!future.isRetriable()) {
throw future.exception();
}
if (remainingToSearch.isEmpty()) {
return result;
} else {
client.awaitMetadataUpdate(timer);
}
} while (timer.notExpired());
throw new TimeoutException("Failed to get offsets by times in " + timer.elapsedMs() + "ms");
}
public Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions, Timer timer) {
return beginningOrEndOffset(partitions, ListOffsetsRequest.EARLIEST_TIMESTAMP, timer);
}
public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions, Timer timer) {
return beginningOrEndOffset(partitions, ListOffsetsRequest.LATEST_TIMESTAMP, timer);
}
private Map<TopicPartition, Long> beginningOrEndOffset(Collection<TopicPartition> partitions,
long timestamp,
Timer timer) {
metadata.addTransientTopics(topicsForPartitions(partitions));
try {
Map<TopicPartition, Long> timestampsToSearch = partitions.stream()
.distinct()
.collect(Collectors.toMap(Function.identity(), tp -> timestamp));
ListOffsetResult result = fetchOffsetsByTimes(timestampsToSearch, timer, false);
return result.fetchedOffsets.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset));
} finally {
metadata.clearTransientTopics();
}
}
/**
* Return the fetched records, empty the record buffer and update the consumed position.
*
* NOTE: returning empty records guarantees the consumed position are NOT updated.
*
* @return The fetched records per partition
* @throws OffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and
* the defaultResetPolicy is NONE
* @throws TopicAuthorizationException If there is TopicAuthorization error in fetchResponse.
*/
public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {
Map<TopicPartition, List<ConsumerRecord<K, V>>> fetched = new HashMap<>();
Queue<CompletedFetch> pausedCompletedFetches = new ArrayDeque<>();
int recordsRemaining = maxPollRecords;
try {
while (recordsRemaining > 0) {
if (nextInLineFetch == null || nextInLineFetch.isConsumed) {
CompletedFetch records = completedFetches.peek();
if (records == null) break;
if (records.notInitialized()) {
try {
nextInLineFetch = initializeCompletedFetch(records);
} catch (Exception e) {
// Remove a completedFetch upon a parse with exception if (1) it contains no records, and
// (2) there are no fetched records with actual content preceding this exception.
// The first condition ensures that the completedFetches is not stuck with the same completedFetch
// in cases such as the TopicAuthorizationException, and the second condition ensures that no
// potential data loss due to an exception in a following record.
FetchResponseData.PartitionData partition = records.partitionData;
if (fetched.isEmpty() && FetchResponse.recordsOrFail(partition).sizeInBytes() == 0) {
completedFetches.poll();
}
throw e;
}
} else {
nextInLineFetch = records;
}
completedFetches.poll();
} else if (subscriptions.isPaused(nextInLineFetch.partition)) {
// when the partition is paused we add the records back to the completedFetches queue instead of draining
// them so that they can be returned on a subsequent poll if the partition is resumed at that time
log.debug("Skipping fetching records for assigned partition {} because it is paused", nextInLineFetch.partition);
pausedCompletedFetches.add(nextInLineFetch);
nextInLineFetch = null;
} else {
List<ConsumerRecord<K, V>> records = fetchRecords(nextInLineFetch, recordsRemaining);
if (!records.isEmpty()) {
TopicPartition partition = nextInLineFetch.partition;
List<ConsumerRecord<K, V>> currentRecords = fetched.get(partition);
if (currentRecords == null) {
fetched.put(partition, records);
} else {
// this case shouldn't usually happen because we only send one fetch at a time per partition,
// but it might conceivably happen in some rare cases (such as partition leader changes).
// we have to copy to a new list because the old one may be immutable
List<ConsumerRecord<K, V>> newRecords = new ArrayList<>(records.size() + currentRecords.size());
newRecords.addAll(currentRecords);
newRecords.addAll(records);
fetched.put(partition, newRecords);
}
recordsRemaining -= records.size();
}
}
}
} catch (KafkaException e) {
if (fetched.isEmpty())
throw e;
} finally {
// add any polled completed fetches for paused partitions back to the completed fetches queue to be
// re-evaluated in the next poll
completedFetches.addAll(pausedCompletedFetches);
}
return fetched;
}
private List<ConsumerRecord<K, V>> fetchRecords(CompletedFetch completedFetch, int maxRecords) {
if (!subscriptions.isAssigned(completedFetch.partition)) {
// this can happen when a rebalance happened before fetched records are returned to the consumer's poll call
log.debug("Not returning fetched records for partition {} since it is no longer assigned",
completedFetch.partition);
} else if (!subscriptions.isFetchable(completedFetch.partition)) {
// this can happen when a partition is paused before fetched records are returned to the consumer's
// poll call or if the offset is being reset
log.debug("Not returning fetched records for assigned partition {} since it is no longer fetchable",
completedFetch.partition);
} else {
FetchPosition position = subscriptions.position(completedFetch.partition);
if (position == null) {
throw new IllegalStateException("Missing position for fetchable partition " + completedFetch.partition);
}
if (completedFetch.nextFetchOffset == position.offset) {
List<ConsumerRecord<K, V>> partRecords = completedFetch.fetchRecords(maxRecords);
log.trace("Returning {} fetched records at offset {} for assigned partition {}",
partRecords.size(), position, completedFetch.partition);
if (completedFetch.nextFetchOffset > position.offset) {
FetchPosition nextPosition = new FetchPosition(
completedFetch.nextFetchOffset,
completedFetch.lastEpoch,
position.currentLeader);
log.trace("Update fetching position to {} for partition {}", nextPosition, completedFetch.partition);
subscriptions.position(completedFetch.partition, nextPosition);
}
Long partitionLag = subscriptions.partitionLag(completedFetch.partition, isolationLevel);
if (partitionLag != null)
this.sensors.recordPartitionLag(completedFetch.partition, partitionLag);
Long lead = subscriptions.partitionLead(completedFetch.partition);
if (lead != null) {
this.sensors.recordPartitionLead(completedFetch.partition, lead);
}
return partRecords;
} else {
// these records aren't next in line based on the last consumed position, ignore them
// they must be from an obsolete request
log.debug("Ignoring fetched records for {} at offset {} since the current position is {}",
completedFetch.partition, completedFetch.nextFetchOffset, position);
}
}
log.trace("Draining fetched records for partition {}", completedFetch.partition);
completedFetch.drain();
return emptyList();
}
// Visible for testing
void resetOffsetIfNeeded(TopicPartition partition, OffsetResetStrategy requestedResetStrategy, ListOffsetData offsetData) {
FetchPosition position = new FetchPosition(
offsetData.offset,
Optional.empty(), // This will ensure we skip validation
metadata.currentLeader(partition));
offsetData.leaderEpoch.ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(partition, epoch));
subscriptions.maybeSeekUnvalidated(partition, position, requestedResetStrategy);
}
private void resetOffsetsAsync(Map<TopicPartition, Long> partitionResetTimestamps) {
Map<Node, Map<TopicPartition, ListOffsetsPartition>> timestampsToSearchByNode =
groupListOffsetRequests(partitionResetTimestamps, new HashSet<>());
for (Map.Entry<Node, Map<TopicPartition, ListOffsetsPartition>> entry : timestampsToSearchByNode.entrySet()) {
Node node = entry.getKey();
final Map<TopicPartition, ListOffsetsPartition> resetTimestamps = entry.getValue();
subscriptions.setNextAllowedRetry(resetTimestamps.keySet(), time.milliseconds() + requestTimeoutMs);
RequestFuture<ListOffsetResult> future = sendListOffsetRequest(node, resetTimestamps, false);
future.addListener(new RequestFutureListener<ListOffsetResult>() {
@Override
public void onSuccess(ListOffsetResult result) {
if (!result.partitionsToRetry.isEmpty()) {
subscriptions.requestFailed(result.partitionsToRetry, time.milliseconds() + retryBackoffMs);
metadata.requestUpdate();
}
for (Map.Entry<TopicPartition, ListOffsetData> fetchedOffset : result.fetchedOffsets.entrySet()) {
TopicPartition partition = fetchedOffset.getKey();
ListOffsetData offsetData = fetchedOffset.getValue();
ListOffsetsPartition requestedReset = resetTimestamps.get(partition);
resetOffsetIfNeeded(partition, timestampToOffsetResetStrategy(requestedReset.timestamp()), offsetData);
}
}
@Override
public void onFailure(RuntimeException e) {
subscriptions.requestFailed(resetTimestamps.keySet(), time.milliseconds() + retryBackoffMs);
metadata.requestUpdate();
if (!(e instanceof RetriableException) && !cachedListOffsetsException.compareAndSet(null, e))
log.error("Discarding error in ListOffsetResponse because another error is pending", e);
}
});
}
}
static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersions) {
ApiVersion apiVersion = nodeApiVersions.apiVersion(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
if (apiVersion == null)
return false;
return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
}
/**
* For each partition which needs validation, make an asynchronous request to get the end-offsets for the partition
* with the epoch less than or equal to the epoch the partition last saw.
*
* Requests are grouped by Node for efficiency.
*/
private void validateOffsetsAsync(Map<TopicPartition, FetchPosition> partitionsToValidate) {
final Map<Node, Map<TopicPartition, FetchPosition>> regrouped =
regroupFetchPositionsByLeader(partitionsToValidate);
long nextResetTimeMs = time.milliseconds() + requestTimeoutMs;
regrouped.forEach((node, fetchPositions) -> {
if (node.isEmpty()) {
metadata.requestUpdate();
return;
}
NodeApiVersions nodeApiVersions = apiVersions.get(node.idString());
if (nodeApiVersions == null) {
client.tryConnect(node);
return;
}
if (!hasUsableOffsetForLeaderEpochVersion(nodeApiVersions)) {
log.debug("Skipping validation of fetch offsets for partitions {} since the broker does not " +
"support the required protocol version (introduced in Kafka 2.3)",
fetchPositions.keySet());
for (TopicPartition partition : fetchPositions.keySet()) {
subscriptions.completeValidation(partition);
}
return;
}
subscriptions.setNextAllowedRetry(fetchPositions.keySet(), nextResetTimeMs);
RequestFuture<OffsetForEpochResult> future =
offsetsForLeaderEpochClient.sendAsyncRequest(node, fetchPositions);
future.addListener(new RequestFutureListener<OffsetForEpochResult>() {
@Override
public void onSuccess(OffsetForEpochResult offsetsResult) {
List<SubscriptionState.LogTruncation> truncations = new ArrayList<>();
if (!offsetsResult.partitionsToRetry().isEmpty()) {
subscriptions.setNextAllowedRetry(offsetsResult.partitionsToRetry(), time.milliseconds() + retryBackoffMs);
metadata.requestUpdate();
}
// For each OffsetsForLeader response, check if the end-offset is lower than our current offset
// for the partition. If so, it means we have experienced log truncation and need to reposition
// that partition's offset.
//
// In addition, check whether the returned offset and epoch are valid. If not, then we should reset
// its offset if reset policy is configured, or throw out of range exception.
offsetsResult.endOffsets().forEach((topicPartition, respEndOffset) -> {
FetchPosition requestPosition = fetchPositions.get(topicPartition);
Optional<SubscriptionState.LogTruncation> truncationOpt =
subscriptions.maybeCompleteValidation(topicPartition, requestPosition, respEndOffset);
truncationOpt.ifPresent(truncations::add);
});
if (!truncations.isEmpty()) {
maybeSetOffsetForLeaderException(buildLogTruncationException(truncations));
}
}
@Override
public void onFailure(RuntimeException e) {
subscriptions.requestFailed(fetchPositions.keySet(), time.milliseconds() + retryBackoffMs);
metadata.requestUpdate();
if (!(e instanceof RetriableException)) {
maybeSetOffsetForLeaderException(e);
}
}
});
});
}
private LogTruncationException buildLogTruncationException(List<SubscriptionState.LogTruncation> truncations) {
Map<TopicPartition, OffsetAndMetadata> divergentOffsets = new HashMap<>();
Map<TopicPartition, Long> truncatedFetchOffsets = new HashMap<>();
for (SubscriptionState.LogTruncation truncation : truncations) {
truncation.divergentOffsetOpt.ifPresent(divergentOffset ->
divergentOffsets.put(truncation.topicPartition, divergentOffset));
truncatedFetchOffsets.put(truncation.topicPartition, truncation.fetchPosition.offset);
}
return new LogTruncationException("Detected truncated partitions: " + truncations,
truncatedFetchOffsets, divergentOffsets);
}
private void maybeSetOffsetForLeaderException(RuntimeException e) {
if (!cachedOffsetForLeaderException.compareAndSet(null, e)) {
log.error("Discarding error in OffsetsForLeaderEpoch because another error is pending", e);
}
}
/**
* Search the offsets by target times for the specified partitions.
*
* @param timestampsToSearch the mapping between partitions and target time
* @param requireTimestamps true if we should fail with an UnsupportedVersionException if the broker does
* not support fetching precise timestamps for offsets
* @return A response which can be polled to obtain the corresponding timestamps and offsets.
*/
private RequestFuture<ListOffsetResult> sendListOffsetsRequests(final Map<TopicPartition, Long> timestampsToSearch,
final boolean requireTimestamps) {
final Set<TopicPartition> partitionsToRetry = new HashSet<>();
Map<Node, Map<TopicPartition, ListOffsetsPartition>> timestampsToSearchByNode =
groupListOffsetRequests(timestampsToSearch, partitionsToRetry);
if (timestampsToSearchByNode.isEmpty())
return RequestFuture.failure(new StaleMetadataException());
final RequestFuture<ListOffsetResult> listOffsetRequestsFuture = new RequestFuture<>();
final Map<TopicPartition, ListOffsetData> fetchedTimestampOffsets = new HashMap<>();
final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size());
for (Map.Entry<Node, Map<TopicPartition, ListOffsetsPartition>> entry : timestampsToSearchByNode.entrySet()) {
RequestFuture<ListOffsetResult> future =
sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps);
future.addListener(new RequestFutureListener<ListOffsetResult>() {
@Override
public void onSuccess(ListOffsetResult partialResult) {
synchronized (listOffsetRequestsFuture) {
fetchedTimestampOffsets.putAll(partialResult.fetchedOffsets);
partitionsToRetry.addAll(partialResult.partitionsToRetry);
if (remainingResponses.decrementAndGet() == 0 && !listOffsetRequestsFuture.isDone()) {
ListOffsetResult result = new ListOffsetResult(fetchedTimestampOffsets, partitionsToRetry);
listOffsetRequestsFuture.complete(result);
}
}
}
@Override
public void onFailure(RuntimeException e) {
synchronized (listOffsetRequestsFuture) {
if (!listOffsetRequestsFuture.isDone())
listOffsetRequestsFuture.raise(e);
}
}
});
}
return listOffsetRequestsFuture;
}
/**
* Groups timestamps to search by node for topic partitions in `timestampsToSearch` that have
* leaders available. Topic partitions from `timestampsToSearch` that do not have their leader
* available are added to `partitionsToRetry`
* @param timestampsToSearch The mapping from partitions ot the target timestamps
* @param partitionsToRetry A set of topic partitions that will be extended with partitions
* that need metadata update or re-connect to the leader.
*/
private Map<Node, Map<TopicPartition, ListOffsetsPartition>> groupListOffsetRequests(
Map<TopicPartition, Long> timestampsToSearch,
Set<TopicPartition> partitionsToRetry) {
final Map<TopicPartition, ListOffsetsPartition> partitionDataMap = new HashMap<>();
for (Map.Entry<TopicPartition, Long> entry: timestampsToSearch.entrySet()) {
TopicPartition tp = entry.getKey();
Long offset = entry.getValue();
Metadata.LeaderAndEpoch leaderAndEpoch = metadata.currentLeader(tp);
if (!leaderAndEpoch.leader.isPresent()) {
log.debug("Leader for partition {} is unknown for fetching offset {}", tp, offset);
metadata.requestUpdate();
partitionsToRetry.add(tp);
} else {
Node leader = leaderAndEpoch.leader.get();
if (client.isUnavailable(leader)) {
client.maybeThrowAuthFailure(leader);
// The connection has failed and we need to await the backoff period before we can
// try again. No need to request a metadata update since the disconnect will have
// done so already.
log.debug("Leader {} for partition {} is unavailable for fetching offset until reconnect backoff expires",
leader, tp);
partitionsToRetry.add(tp);
} else {
int currentLeaderEpoch = leaderAndEpoch.epoch.orElse(ListOffsetsResponse.UNKNOWN_EPOCH);
partitionDataMap.put(tp, new ListOffsetsPartition()
.setPartitionIndex(tp.partition())
.setTimestamp(offset)
.setCurrentLeaderEpoch(currentLeaderEpoch));
}
}
}
return regroupPartitionMapByNode(partitionDataMap);
}
/**
* Send the ListOffsetRequest to a specific broker for the partitions and target timestamps.
*
* @param node The node to send the ListOffsetRequest to.
* @param timestampsToSearch The mapping from partitions to the target timestamps.
* @param requireTimestamp True if we require a timestamp in the response.
* @return A response which can be polled to obtain the corresponding timestamps and offsets.
*/
private RequestFuture<ListOffsetResult> sendListOffsetRequest(final Node node,
final Map<TopicPartition, ListOffsetsPartition> timestampsToSearch,
boolean requireTimestamp) {
ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder
.forConsumer(requireTimestamp, isolationLevel)
.setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(timestampsToSearch));
log.debug("Sending ListOffsetRequest {} to broker {}", builder, node);
return client.send(node, builder)
.compose(new RequestFutureAdapter<ClientResponse, ListOffsetResult>() {
@Override
public void onSuccess(ClientResponse response, RequestFuture<ListOffsetResult> future) {
ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody();
log.trace("Received ListOffsetResponse {} from broker {}", lor, node);
handleListOffsetResponse(lor, future);
}
});
}
/**
* Callback for the response of the list offset call above.
* @param listOffsetsResponse The response from the server.
* @param future The future to be completed when the response returns. Note that any partition-level errors will
* generally fail the entire future result. The one exception is UNSUPPORTED_FOR_MESSAGE_FORMAT,