forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
CommitRequestManager.java
499 lines (444 loc) · 23.8 KB
/
CommitRequestManager.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
/*
* 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.ClientResponse;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;
public class CommitRequestManager implements RequestManager {
// TODO: current in ConsumerConfig but inaccessible in the internal package.
private static final String THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED = "internal.throw.on.fetch.stable.offset.unsupported";
// TODO: We will need to refactor the subscriptionState
private final SubscriptionState subscriptionState;
private final Logger log;
private final Optional<AutoCommitState> autoCommitState;
private final CoordinatorRequestManager coordinatorRequestManager;
private final GroupState groupState;
private final long retryBackoffMs;
private final boolean throwOnFetchStableOffsetUnsupported;
final PendingRequests pendingRequests;
public CommitRequestManager(
final Time time,
final LogContext logContext,
final SubscriptionState subscriptionState,
final ConsumerConfig config,
final CoordinatorRequestManager coordinatorRequestManager,
final GroupState groupState) {
Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets");
this.log = logContext.logger(getClass());
this.pendingRequests = new PendingRequests();
if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
final long autoCommitInterval =
Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval));
} else {
this.autoCommitState = Optional.empty();
}
this.coordinatorRequestManager = coordinatorRequestManager;
this.groupState = groupState;
this.subscriptionState = subscriptionState;
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.throwOnFetchStableOffsetUnsupported = config.getBoolean(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED);
}
/**
* Poll for the {@link OffsetFetchRequest} and {@link OffsetCommitRequest} request if there's any. The function will
* also try to autocommit the offsets, if feature is enabled.
*/
@Override
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
maybeAutoCommit(this.subscriptionState.allConsumed());
if (!pendingRequests.hasUnsentRequests()) {
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList());
}
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE,
Collections.unmodifiableList(pendingRequests.drain(currentTimeMs)));
}
public void maybeAutoCommit(final Map<TopicPartition, OffsetAndMetadata> offsets) {
if (!autoCommitState.isPresent()) {
return;
}
AutoCommitState autocommit = autoCommitState.get();
if (!autocommit.canSendAutocommit()) {
return;
}
sendAutoCommit(offsets);
autocommit.resetTimer();
autocommit.setInflightCommitStatus(true);
}
/**
* Handles {@link org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent}. It creates an
* {@link OffsetCommitRequestState} and enqueue it to send later.
*/
public CompletableFuture<ClientResponse> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) {
return pendingRequests.addOffsetCommitRequest(offsets);
}
/**
* Handles {@link org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent}. It creates an
* {@link OffsetFetchRequestState} and enqueue it to send later.
*/
public CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> addOffsetFetchRequest(final Set<TopicPartition> partitions) {
return pendingRequests.addOffsetFetchRequest(partitions);
}
public void updateAutoCommitTimer(final long currentTimeMs) {
this.autoCommitState.ifPresent(t -> t.ack(currentTimeMs));
}
// Visible for testing
List<OffsetFetchRequestState> unsentOffsetFetchRequests() {
return pendingRequests.unsentOffsetFetches;
}
// Visible for testing
Queue<OffsetCommitRequestState> unsentOffsetCommitRequests() {
return pendingRequests.unsentOffsetCommits;
}
// Visible for testing
CompletableFuture<ClientResponse> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) {
log.debug("Enqueuing autocommit offsets: {}", allConsumedOffsets);
return this.addOffsetCommitRequest(allConsumedOffsets)
.whenComplete((response, throwable) -> {
this.autoCommitState.ifPresent(autoCommitState -> autoCommitState.setInflightCommitStatus(false));
if (throwable == null) {
log.debug("Completed asynchronous auto-commit of offsets {}", allConsumedOffsets);
}
})
.exceptionally(t -> {
if (t instanceof RetriableCommitFailedException) {
log.debug("Asynchronous auto-commit of offsets {} failed due to retriable error: {}", allConsumedOffsets, t);
} else {
log.warn("Asynchronous auto-commit of offsets {} failed: {}", allConsumedOffsets, t.getMessage());
}
return null;
});
}
private class OffsetCommitRequestState {
private final Map<TopicPartition, OffsetAndMetadata> offsets;
private final String groupId;
private final GroupState.Generation generation;
private final String groupInstanceId;
private final NetworkClientDelegate.FutureCompletionHandler future;
public OffsetCommitRequestState(final Map<TopicPartition, OffsetAndMetadata> offsets,
final String groupId,
final String groupInstanceId,
final GroupState.Generation generation) {
this.offsets = offsets;
this.future = new NetworkClientDelegate.FutureCompletionHandler();
this.groupId = groupId;
this.generation = generation;
this.groupInstanceId = groupInstanceId;
}
public CompletableFuture<ClientResponse> future() {
return future.future();
}
public NetworkClientDelegate.UnsentRequest toUnsentRequest() {
Map<String, OffsetCommitRequestData.OffsetCommitRequestTopic> requestTopicDataMap = new HashMap<>();
for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
TopicPartition topicPartition = entry.getKey();
OffsetAndMetadata offsetAndMetadata = entry.getValue();
OffsetCommitRequestData.OffsetCommitRequestTopic topic = requestTopicDataMap
.getOrDefault(topicPartition.topic(),
new OffsetCommitRequestData.OffsetCommitRequestTopic()
.setName(topicPartition.topic())
);
topic.partitions().add(new OffsetCommitRequestData.OffsetCommitRequestPartition()
.setPartitionIndex(topicPartition.partition())
.setCommittedOffset(offsetAndMetadata.offset())
.setCommittedLeaderEpoch(offsetAndMetadata.leaderEpoch().orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
.setCommittedMetadata(offsetAndMetadata.metadata())
);
requestTopicDataMap.put(topicPartition.topic(), topic);
}
OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(
new OffsetCommitRequestData()
.setGroupId(this.groupId)
.setGenerationId(generation.generationId)
.setMemberId(generation.memberId)
.setGroupInstanceId(groupInstanceId)
.setTopics(new ArrayList<>(requestTopicDataMap.values())));
return new NetworkClientDelegate.UnsentRequest(
builder,
coordinatorRequestManager.coordinator(),
future);
}
}
private class OffsetFetchRequestState extends RequestState {
public final Set<TopicPartition> requestedPartitions;
public final GroupState.Generation requestedGeneration;
public CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future;
public OffsetFetchRequestState(final Set<TopicPartition> partitions,
final GroupState.Generation generation,
final long retryBackoffMs) {
super(retryBackoffMs);
this.requestedPartitions = partitions;
this.requestedGeneration = generation;
this.future = new CompletableFuture<>();
}
public boolean sameRequest(final OffsetFetchRequestState request) {
return Objects.equals(requestedGeneration, request.requestedGeneration) && requestedPartitions.equals(request.requestedPartitions);
}
public NetworkClientDelegate.UnsentRequest toUnsentRequest(final long currentTimeMs) {
OffsetFetchRequest.Builder builder = new OffsetFetchRequest.Builder(
groupState.groupId,
true,
new ArrayList<>(this.requestedPartitions),
throwOnFetchStableOffsetUnsupported);
NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest(
builder,
coordinatorRequestManager.coordinator());
unsentRequest.future().whenComplete((r, t) -> {
onResponse(currentTimeMs, (OffsetFetchResponse) r.responseBody());
});
return unsentRequest;
}
public void onResponse(
final long currentTimeMs,
final OffsetFetchResponse response) {
Errors responseError = response.groupLevelError(groupState.groupId);
if (responseError != Errors.NONE) {
onFailure(currentTimeMs, responseError);
return;
}
onSuccess(currentTimeMs, response);
}
private void onFailure(final long currentTimeMs,
final Errors responseError) {
log.debug("Offset fetch failed: {}", responseError.message());
// TODO: should we retry on COORDINATOR_NOT_AVAILABLE as well ?
if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
retry(currentTimeMs);
} else if (responseError == Errors.NOT_COORDINATOR) {
// re-discover the coordinator and retry
coordinatorRequestManager.markCoordinatorUnknown(responseError.message(), Time.SYSTEM.milliseconds());
retry(currentTimeMs);
} else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
future.completeExceptionally(GroupAuthorizationException.forGroupId(groupState.groupId));
} else {
future.completeExceptionally(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
}
}
private void retry(final long currentTimeMs) {
onFailedAttempt(currentTimeMs);
onSendAttempt(currentTimeMs);
pendingRequests.addOffsetFetchRequest(this);
}
private void onSuccess(final long currentTimeMs,
final OffsetFetchResponse response) {
Set<String> unauthorizedTopics = null;
Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
response.partitionDataMap(groupState.groupId);
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(responseData.size());
Set<TopicPartition> unstableTxnOffsetTopicPartitions = new HashSet<>();
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : responseData.entrySet()) {
TopicPartition tp = entry.getKey();
OffsetFetchResponse.PartitionData partitionData = entry.getValue();
if (partitionData.hasError()) {
Errors error = partitionData.error;
log.debug("Failed to fetch offset for partition {}: {}", tp, error.message());
if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
future.completeExceptionally(new KafkaException("Topic or Partition " + tp + " does " +
"not " +
"exist"));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
if (unauthorizedTopics == null) {
unauthorizedTopics = new HashSet<>();
}
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.UNSTABLE_OFFSET_COMMIT) {
unstableTxnOffsetTopicPartitions.add(tp);
} else {
future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " +
"response for partition " + tp + ": " + error.message()));
return;
}
} else if (partitionData.offset >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch);
// if there's no committed offset, record as null
offsets.put(tp, new OffsetAndMetadata(partitionData.offset, partitionData.leaderEpoch, partitionData.metadata));
} else {
log.info("Found no committed offset for partition {}", tp);
offsets.put(tp, null);
}
}
if (unauthorizedTopics != null) {
future.completeExceptionally(new TopicAuthorizationException(unauthorizedTopics));
} else if (!unstableTxnOffsetTopicPartitions.isEmpty()) {
// TODO: Optimization question: Do we need to retry all partitions upon a single partition error?
log.info("The following partitions still have unstable offsets " +
"which are not cleared on the broker side: {}" +
", this could be either " +
"transactional offsets waiting for completion, or " +
"normal offsets waiting for replication after appending to local log", unstableTxnOffsetTopicPartitions);
retry(currentTimeMs);
} else {
future.complete(offsets);
}
}
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> chainFuture(final CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
return this.future.whenComplete((r, t) -> {
if (t != null) {
future.completeExceptionally(t);
} else {
future.complete(r);
}
});
}
}
/**
* <p>This is used to stage the unsent {@link OffsetCommitRequestState} and {@link OffsetFetchRequestState}.
* <li>unsentOffsetCommits holds the offset commit requests that have not been sent out</>
* <li>unsentOffsetFetches holds the offset fetch requests that have not been sent out</li>
* <li>inflightOffsetFetches holds the offset fetch requests that have been sent out but incompleted</>.
*
* {@code addOffsetFetchRequest} dedupes the requests to avoid sending the same requests.
*/
class PendingRequests {
// Queue is used to ensure the sequence of commit
Queue<OffsetCommitRequestState> unsentOffsetCommits = new LinkedList<>();
List<OffsetFetchRequestState> unsentOffsetFetches = new ArrayList<>();
List<OffsetFetchRequestState> inflightOffsetFetches = new ArrayList<>();
public boolean hasUnsentRequests() {
return !unsentOffsetCommits.isEmpty() || !unsentOffsetFetches.isEmpty();
}
public CompletableFuture<ClientResponse> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) {
// TODO: Dedupe committing the same offsets to the same partitions
OffsetCommitRequestState request = new OffsetCommitRequestState(
offsets,
groupState.groupId,
groupState.groupInstanceId.orElse(null),
groupState.generation);
unsentOffsetCommits.add(request);
return request.future();
}
/**
* <p>Adding an offset fetch request to the outgoing buffer. If the same request was made, we chain the future
* to the existing one.
*
* <p>If the request is new, it invokes a callback to remove itself from the {@code inflightOffsetFetches}
* upon completion.</>
*/
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> addOffsetFetchRequest(final OffsetFetchRequestState request) {
Optional<OffsetFetchRequestState> dupe =
unsentOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny();
Optional<OffsetFetchRequestState> inflight =
inflightOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny();
if (dupe.isPresent() || inflight.isPresent()) {
log.info("Duplicated OffsetFetchRequest: " + request.requestedPartitions);
dupe.orElseGet(() -> inflight.get()).chainFuture(request.future);
} else {
// remove the request from the outbound buffer: inflightOffsetFetches
request.future.whenComplete((r, t) -> {
if (!inflightOffsetFetches.remove(request)) {
log.warn("A duplicated, inflight, request was identified, but unable to find it in the " +
"outbound buffer:" + request);
}
});
this.unsentOffsetFetches.add(request);
}
return request.future;
}
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> addOffsetFetchRequest(final Set<TopicPartition> partitions) {
OffsetFetchRequestState request = new OffsetFetchRequestState(
partitions,
groupState.generation,
retryBackoffMs);
return addOffsetFetchRequest(request);
}
/**
* Clear {@code unsentOffsetCommits} and moves all the sendable request in {@code unsentOffsetFetches} to the
* {@code inflightOffsetFetches} to bookkeep all of the inflight requests.
*
* Note: Sendable requests are determined by their timer as we are expecting backoff on failed attempt. See
* {@link RequestState}.
**/
public List<NetworkClientDelegate.UnsentRequest> drain(final long currentTimeMs) {
List<NetworkClientDelegate.UnsentRequest> unsentRequests = new ArrayList<>();
// Add all unsent offset commit requests to the unsentRequests list
unsentRequests.addAll(
unsentOffsetCommits.stream()
.map(OffsetCommitRequestState::toUnsentRequest)
.collect(Collectors.toList()));
// Partition the unsent offset fetch requests into sendable and non-sendable lists
Map<Boolean, List<OffsetFetchRequestState>> partitionedBySendability =
unsentOffsetFetches.stream()
.collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs)));
// Add all sendable offset fetch requests to the unsentRequests list and to the inflightOffsetFetches list
for (OffsetFetchRequestState request : partitionedBySendability.get(true)) {
request.onSendAttempt(currentTimeMs);
unsentRequests.add(request.toUnsentRequest(currentTimeMs));
inflightOffsetFetches.add(request);
}
// Clear the unsent offset commit and fetch lists and add all non-sendable offset fetch requests to the unsentOffsetFetches list
unsentOffsetCommits.clear();
unsentOffsetFetches.clear();
unsentOffsetFetches.addAll(partitionedBySendability.get(false));
return Collections.unmodifiableList(unsentRequests);
}
}
/**
* Encapsulates the state of auto-committing and manages the auto-commit timer.
*/
private static class AutoCommitState {
private final Timer timer;
private final long autoCommitInterval;
private boolean hasInflightCommit;
public AutoCommitState(
final Time time,
final long autoCommitInterval) {
this.autoCommitInterval = autoCommitInterval;
this.timer = time.timer(autoCommitInterval);
this.hasInflightCommit = false;
}
public boolean canSendAutocommit() {
return !this.hasInflightCommit && this.timer.isExpired();
}
public void resetTimer() {
this.timer.reset(autoCommitInterval);
}
public void ack(final long currentTimeMs) {
this.timer.update(currentTimeMs);
}
public void setInflightCommitStatus(final boolean inflightCommitStatus) {
this.hasInflightCommit = inflightCommitStatus;
}
}
}