-
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 all 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 |
---|---|---|
|
@@ -38,8 +38,8 @@ | |
* Whether there is an existing coordinator. | ||
* Whether there is an inflight request. | ||
* Whether the backoff timer has expired. | ||
* The {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult} contains either a wait timer | ||
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. just cleaning up long references |
||
* or a singleton list of {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest}. | ||
* The {@link NetworkClientDelegate.PollResult} contains either a wait timer | ||
* or a singleton list of {@link NetworkClientDelegate.UnsentRequest}. | ||
* <p/> | ||
* The {@link FindCoordinatorRequest} will be handled by the {@link #onResponse(long, FindCoordinatorResponse)} callback, which | ||
* subsequently invokes {@code onResponse} to handle the exception and response. Note that the coordinator node will be | ||
|
@@ -86,7 +86,7 @@ public CoordinatorRequestManager( | |
* Note that this method does not involve any actual network IO, and it only determines if we need to send a new request or not. | ||
* | ||
* @param currentTimeMs current time in ms. | ||
* @return {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult}. This will not be {@code null}. | ||
* @return {@link NetworkClientDelegate.PollResult}. This will not be {@code null}. | ||
*/ | ||
@Override | ||
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -43,6 +43,7 @@ | |
import java.util.Queue; | ||
import java.util.Set; | ||
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. | ||
|
@@ -211,17 +212,19 @@ public static class UnsentRequest { | |
private Optional<Node> node; // empty if random node can be chosen | ||
private Timer timer; | ||
|
||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder, final Optional<Node> node) { | ||
this(requestBuilder, node, new FutureCompletionHandler()); | ||
} | ||
|
||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder, | ||
final Optional<Node> node, | ||
final FutureCompletionHandler handler) { | ||
final Optional<Node> node) { | ||
Objects.requireNonNull(requestBuilder); | ||
this.requestBuilder = requestBuilder; | ||
this.node = node; | ||
this.handler = handler; | ||
this.handler = new FutureCompletionHandler(); | ||
} | ||
|
||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder, | ||
final Optional<Node> node, | ||
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) { | ||
|
@@ -263,10 +266,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) { | ||
|
@@ -280,5 +279,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.
Could this be private? Ditto below.
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.
Sorry - I left out commit. Fixing those obvious mistakes right away.
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.
Sorry - leaving this protected as it is used in one of the test to verify there's no more unsent request.