-
Notifications
You must be signed in to change notification settings - Fork 13.7k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-14960: TopicMetadata request manager #14386
Changes from 24 commits
33a221e
a350729
76d5c96
d33188b
ceb8694
3b7400b
1ac4148
875abb0
a8be7ba
227d9b6
ae44539
4b2a7f7
74b4960
c89958c
d4e874a
b86bc87
d236f1b
09463d2
6a00f7e
381730f
530535d
8a0687c
26d59dd
d1bada5
ffc66e9
6c06137
ec600b3
403e122
269cbcb
179cd5d
628cb4c
6473597
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,7 +16,6 @@ | |
*/ | ||
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; | ||
|
@@ -27,7 +26,6 @@ | |
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; | ||
|
@@ -91,7 +89,7 @@ public CommitRequestManager( | |
} | ||
|
||
/** | ||
* Poll for the {@link OffsetFetchRequest} and {@link OffsetCommitRequest} request if there's any. The function will | ||
* Poll for the {@link OffsetFetchRequest} and {@link org.apache.kafka.common.requests.OffsetCommitRequest} request if there's any. The function will | ||
* also try to autocommit the offsets, if feature is enabled. | ||
*/ | ||
@Override | ||
|
@@ -106,6 +104,7 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { | |
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList()); | ||
} | ||
|
||
pendingRequests.inflightOffsetFetches.forEach(System.out::println); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this intended? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤦 |
||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, | ||
Collections.unmodifiableList(pendingRequests.drain(currentTimeMs))); | ||
} | ||
|
@@ -127,9 +126,9 @@ public void maybeAutoCommit(final Map<TopicPartition, OffsetAndMetadata> offsets | |
|
||
/** | ||
* Handles {@link org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent}. It creates an | ||
* {@link OffsetCommitRequestState} and enqueue it to send later. | ||
* {@link OffsetCommitRequest} and enqueue it to send later. | ||
*/ | ||
public CompletableFuture<ClientResponse> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) { | ||
public CompletableFuture<Void> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) { | ||
return pendingRequests.addOffsetCommitRequest(offsets); | ||
} | ||
|
||
|
@@ -145,19 +144,18 @@ 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() { | ||
Queue<OffsetCommitRequest> unsentOffsetCommitRequests() { | ||
return pendingRequests.unsentOffsetCommits; | ||
} | ||
|
||
// Visible for testing | ||
CompletableFuture<ClientResponse> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) { | ||
CompletableFuture<Void> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) { | ||
log.debug("Enqueuing autocommit offsets: {}", allConsumedOffsets); | ||
return this.addOffsetCommitRequest(allConsumedOffsets) | ||
.whenComplete((response, throwable) -> { | ||
|
@@ -177,28 +175,24 @@ CompletableFuture<ClientResponse> sendAutoCommit(final Map<TopicPartition, Offse | |
}); | ||
} | ||
|
||
private class OffsetCommitRequestState { | ||
private class OffsetCommitRequest { | ||
private final Map<TopicPartition, OffsetAndMetadata> offsets; | ||
private final String groupId; | ||
private final GroupState.Generation generation; | ||
private final String groupInstanceId; | ||
private final NetworkClientDelegate.FutureCompletionHandler future; | ||
private final CompletableFuture<Void> future; | ||
|
||
public OffsetCommitRequestState(final Map<TopicPartition, OffsetAndMetadata> offsets, | ||
final String groupId, | ||
final String groupInstanceId, | ||
final GroupState.Generation generation) { | ||
public OffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets, | ||
final String groupId, | ||
final String groupInstanceId, | ||
final GroupState.Generation generation) { | ||
this.offsets = offsets; | ||
this.future = new NetworkClientDelegate.FutureCompletionHandler(); | ||
this.future = new CompletableFuture<>(); | ||
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()) { | ||
|
@@ -220,7 +214,7 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { | |
requestTopicDataMap.put(topicPartition.topic(), topic); | ||
} | ||
|
||
OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder( | ||
org.apache.kafka.common.requests.OffsetCommitRequest.Builder builder = new org.apache.kafka.common.requests.OffsetCommitRequest.Builder( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could we import the package? |
||
new OffsetCommitRequestData() | ||
.setGroupId(this.groupId) | ||
.setGenerationIdOrMemberEpoch(generation.generationId) | ||
|
@@ -230,15 +224,20 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { | |
return new NetworkClientDelegate.UnsentRequest( | ||
builder, | ||
coordinatorRequestManager.coordinator(), | ||
future); | ||
(response, throwable) -> { | ||
if (throwable == null) { | ||
this.future.complete(null); | ||
} else { | ||
this.future.completeExceptionally(throwable); | ||
} | ||
}); | ||
} | ||
} | ||
|
||
private class OffsetFetchRequestState extends RequestState { | ||
public final Set<TopicPartition> requestedPartitions; | ||
public final GroupState.Generation requestedGeneration; | ||
public CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future; | ||
|
||
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could this be private? |
||
public OffsetFetchRequestState(final Set<TopicPartition> partitions, | ||
final GroupState.Generation generation, | ||
final long retryBackoffMs, | ||
|
@@ -259,13 +258,10 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest(final long currentTim | |
true, | ||
new ArrayList<>(this.requestedPartitions), | ||
throwOnFetchStableOffsetUnsupported); | ||
NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest( | ||
return new NetworkClientDelegate.UnsentRequest( | ||
builder, | ||
coordinatorRequestManager.coordinator()); | ||
unsentRequest.future().whenComplete((r, t) -> { | ||
onResponse(currentTimeMs, (OffsetFetchResponse) r.responseBody()); | ||
}); | ||
return unsentRequest; | ||
coordinatorRequestManager.coordinator(), | ||
(r, t) -> onResponse(currentTimeMs, (OffsetFetchResponse) r.responseBody())); | ||
} | ||
|
||
public void onResponse( | ||
|
@@ -298,8 +294,8 @@ private void onFailure(final long currentTimeMs, | |
} | ||
|
||
private void retry(final long currentTimeMs) { | ||
onFailedAttempt(currentTimeMs); | ||
onSendAttempt(currentTimeMs); | ||
this.onFailedAttempt(currentTimeMs); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. currentTimeMs is when the request is sent, not when the response is receive. Could we pass along There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for catching this. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Addressed, see line #259 |
||
this.onSendAttempt(currentTimeMs); | ||
pendingRequests.addOffsetFetchRequest(this); | ||
} | ||
|
||
|
@@ -359,12 +355,12 @@ private void onSuccess(final long currentTimeMs, | |
} | ||
} | ||
|
||
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> chainFuture(final CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future) { | ||
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> chainFuture(final CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> otherFuture) { | ||
return this.future.whenComplete((r, t) -> { | ||
if (t != null) { | ||
future.completeExceptionally(t); | ||
otherFuture.completeExceptionally(t); | ||
} else { | ||
future.complete(r); | ||
otherFuture.complete(r); | ||
} | ||
}); | ||
} | ||
|
@@ -381,41 +377,41 @@ public String toString() { | |
} | ||
|
||
/** | ||
* <p>This is used to stage the unsent {@link OffsetCommitRequestState} and {@link OffsetFetchRequestState}. | ||
* <p>This is used to stage the unsent {@link OffsetCommitRequest} 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</>. | ||
* | ||
* <li>inflightOffsetFetches holds the offset fetch requests that have been sent out but not completed</>. | ||
* <p> | ||
* {@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<>(); | ||
Queue<OffsetCommitRequest> unsentOffsetCommits = new LinkedList<>(); | ||
List<OffsetFetchRequestState> unsentOffsetFetches = new ArrayList<>(); | ||
List<OffsetFetchRequestState> inflightOffsetFetches = new ArrayList<>(); | ||
|
||
public boolean hasUnsentRequests() { | ||
boolean hasUnsentRequests() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could this be private? Ditto below. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry - I left out commit. Fixing those obvious mistakes right away. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry - leaving this protected as it is used in one of the test to verify there's no more unsent request. |
||
return !unsentOffsetCommits.isEmpty() || !unsentOffsetFetches.isEmpty(); | ||
} | ||
|
||
public CompletableFuture<ClientResponse> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) { | ||
CompletableFuture<Void> addOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) { | ||
// TODO: Dedupe committing the same offsets to the same partitions | ||
OffsetCommitRequestState request = new OffsetCommitRequestState( | ||
OffsetCommitRequest request = new OffsetCommitRequest( | ||
offsets, | ||
groupState.groupId, | ||
groupState.groupInstanceId.orElse(null), | ||
groupState.generation); | ||
unsentOffsetCommits.add(request); | ||
return request.future(); | ||
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>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.</> | ||
* <p>If the request is new, it invokes a callback to remove itself from the {@code inflightOffsetFetches} | ||
* upon completion.</> | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: wrong tag </> and other unclosed ones above |
||
*/ | ||
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> addOffsetFetchRequest(final OffsetFetchRequestState request) { | ||
Optional<OffsetFetchRequestState> dupe = | ||
|
@@ -451,17 +447,17 @@ private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> addOffsetFetch | |
/** | ||
* Clear {@code unsentOffsetCommits} and moves all the sendable request in {@code unsentOffsetFetches} to the | ||
* {@code inflightOffsetFetches} to bookkeep all of the inflight requests. | ||
* | ||
* <p> | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. was the intention here to add an extra line in-between? should be There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 😓 - must be the residual from some edits.. |
||
* 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> 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) | ||
.map(OffsetCommitRequest::toUnsentRequest) | ||
.collect(Collectors.toList())); | ||
|
||
// Partition the unsent offset fetch requests into sendable and non-sendable lists | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -40,6 +40,7 @@ | |
import java.util.Optional; | ||
import java.util.Queue; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.function.BiConsumer; | ||
|
||
/** | ||
* A wrapper around the {@link org.apache.kafka.clients.NetworkClient} to handle network poll and send operations. | ||
|
@@ -203,16 +204,17 @@ public static class UnsentRequest { | |
private Timer timer; | ||
|
||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder, final Optional<Node> node) { | ||
this(requestBuilder, node, new FutureCompletionHandler()); | ||
Objects.requireNonNull(requestBuilder); | ||
this.requestBuilder = requestBuilder; | ||
this.node = node; | ||
this.handler = new FutureCompletionHandler(); | ||
} | ||
|
||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder, | ||
final Optional<Node> node, | ||
final FutureCompletionHandler handler) { | ||
Objects.requireNonNull(requestBuilder); | ||
this.requestBuilder = requestBuilder; | ||
this.node = node; | ||
this.handler = handler; | ||
final BiConsumer<ClientResponse, Throwable> callback) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This constructor seems unused? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is according to my IDE. I think it is used in CommitRequestManager (line 219 and 256) and TopicMetadataRequestManger (line 145) |
||
this(requestBuilder, node); | ||
this.handler.future.whenComplete(callback); | ||
} | ||
|
||
public void setTimer(final Time time, final long requestTimeoutMs) { | ||
|
@@ -254,10 +256,6 @@ public void onFailure(final RuntimeException e) { | |
future.completeExceptionally(e); | ||
} | ||
|
||
public CompletableFuture<ClientResponse> future() { | ||
return future; | ||
} | ||
|
||
@Override | ||
public void onComplete(final ClientResponse response) { | ||
if (response.authenticationException() != null) { | ||
|
@@ -271,5 +269,4 @@ public void onComplete(final ClientResponse response) { | |
} | ||
} | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to spell out the package name for OffsetCommitRequest? It doesn't seem this is done consistently.