-
Couldn't load subscription status.
- Fork 14.8k
KAFKA-7109: Close fetch sessions on close of consumer #12590
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
Changes from all commits
e3d3e1c
798dfa7
459b458
0786164
ec6e48c
ff731f1
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 |
|---|---|---|
|
|
@@ -104,6 +104,7 @@ | |
| import java.util.Queue; | ||
| import java.util.Set; | ||
| 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.function.Function; | ||
|
|
@@ -159,7 +160,7 @@ public class Fetcher<K, V> implements Closeable { | |
| private final Set<Integer> nodesWithPendingFetchRequests; | ||
| private final ApiVersions apiVersions; | ||
| private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1); | ||
|
|
||
| private final AtomicBoolean isClosed = new AtomicBoolean(false); | ||
| private CompletedFetch nextInLineFetch = null; | ||
|
|
||
| public Fetcher(LogContext logContext, | ||
|
|
@@ -253,25 +254,7 @@ public synchronized int sendFetches() { | |
| for (Map.Entry<Node, FetchSessionHandler.FetchRequestData> entry : fetchRequestMap.entrySet()) { | ||
| final Node fetchTarget = entry.getKey(); | ||
| final FetchSessionHandler.FetchRequestData data = entry.getValue(); | ||
| final short maxVersion; | ||
| if (!data.canUseTopicIds()) { | ||
| maxVersion = (short) 12; | ||
| } else { | ||
| maxVersion = ApiKeys.FETCH.latestVersion(); | ||
| } | ||
| final FetchRequest.Builder request = FetchRequest.Builder | ||
| .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend()) | ||
| .isolationLevel(isolationLevel) | ||
| .setMaxBytes(this.maxBytes) | ||
| .metadata(data.metadata()) | ||
| .removed(data.toForget()) | ||
| .replaced(data.toReplace()) | ||
| .rackId(clientRackId); | ||
|
|
||
| if (log.isDebugEnabled()) { | ||
| log.debug("Sending {} {} to broker {}", isolationLevel, data.toString(), fetchTarget); | ||
| } | ||
| RequestFuture<ClientResponse> future = client.send(fetchTarget, request); | ||
| final RequestFuture<ClientResponse> future = sendFetchRequestToNode(data, fetchTarget); | ||
| // 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 | ||
|
|
@@ -447,6 +430,33 @@ private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builde | |
| return client.send(node, request); | ||
| } | ||
|
|
||
| /** | ||
| * Send Fetch Request to Kafka cluster asynchronously. | ||
| * | ||
| * This method is visible for testing. | ||
| * | ||
| * @return A future that indicates result of sent Fetch request | ||
| */ | ||
| private RequestFuture<ClientResponse> sendFetchRequestToNode(final FetchSessionHandler.FetchRequestData requestData, | ||
| final Node fetchTarget) { | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| // Version 12 is the maximum version that could be used without topic IDs. See FetchRequest.json for schema | ||
| // changelog. | ||
| final short maxVersion = requestData.canUseTopicIds() ? ApiKeys.FETCH.latestVersion() : (short) 12; | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| final FetchRequest.Builder request = FetchRequest.Builder | ||
| .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, requestData.toSend()) | ||
| .isolationLevel(isolationLevel) | ||
| .setMaxBytes(this.maxBytes) | ||
| .metadata(requestData.metadata()) | ||
| .removed(requestData.toForget()) | ||
| .replaced(requestData.toReplace()) | ||
| .rackId(clientRackId); | ||
|
|
||
| log.debug("Sending {} {} to broker {}", isolationLevel, requestData, fetchTarget); | ||
showuon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| return client.send(fetchTarget, request); | ||
| } | ||
|
|
||
| private Long offsetResetStrategyTimestamp(final TopicPartition partition) { | ||
| OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); | ||
| if (strategy == OffsetResetStrategy.EARLIEST) | ||
|
|
@@ -1936,11 +1946,77 @@ private Map<String, String> topicPartitionTags(TopicPartition tp) { | |
| } | ||
| } | ||
|
|
||
| // Visible for testing | ||
| void maybeCloseFetchSessions(final Timer timer) { | ||
| final Cluster cluster = metadata.fetch(); | ||
| final List<RequestFuture<ClientResponse>> requestFutures = new ArrayList<>(); | ||
| sessionHandlers.forEach((fetchTargetNodeId, sessionHandler) -> { | ||
| // set the session handler to notify close. This will set the next metadata request to send close message. | ||
| sessionHandler.notifyClose(); | ||
|
|
||
| final int sessionId = sessionHandler.sessionId(); | ||
| // FetchTargetNode may not be available as it may have disconnected the connection. In such cases, we will | ||
| // skip sending the close request. | ||
| final Node fetchTarget = cluster.nodeById(fetchTargetNodeId); | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| if (fetchTarget == null || client.isUnavailable(fetchTarget)) { | ||
|
||
| log.debug("Skip sending close session request to broker {} since it is not reachable", fetchTarget); | ||
| return; | ||
| } | ||
|
|
||
| final RequestFuture<ClientResponse> responseFuture = sendFetchRequestToNode(sessionHandler.newBuilder().build(), fetchTarget); | ||
| responseFuture.addListener(new RequestFutureListener<ClientResponse>() { | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| @Override | ||
| public void onSuccess(ClientResponse value) { | ||
| log.debug("Successfully sent a close message for fetch session: {} to node: {}", sessionId, fetchTarget); | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(RuntimeException e) { | ||
| log.debug("Unable to a close message for fetch session: {} to node: {}. " + | ||
| "This may result in unnecessary fetch sessions at the broker.", sessionId, fetchTarget, e); | ||
| } | ||
| }); | ||
|
|
||
| requestFutures.add(responseFuture); | ||
| }); | ||
|
|
||
| // Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until | ||
| // all requests have received a response. | ||
| while (timer.notExpired() && !requestFutures.stream().allMatch(RequestFuture::isDone)) { | ||
| client.poll(timer, null, true); | ||
| } | ||
|
|
||
| if (!requestFutures.stream().allMatch(RequestFuture::isDone)) { | ||
| // we ran out of time before completing all futures. It is ok since we don't want to block the shutdown | ||
| // here. | ||
| log.debug("All requests couldn't be sent in the specific timeout period {}ms. " + | ||
| "This may result in unnecessary fetch sessions at the broker. Consider increasing the timeout passed for " + | ||
| "KafkaConsumer.close(Duration timeout)", timer.timeoutMs()); | ||
| } | ||
| } | ||
|
|
||
| public void close(final Timer timer) { | ||
| if (!isClosed.compareAndSet(false, true)) { | ||
| log.info("Fetcher {} is already closed.", this); | ||
| return; | ||
| } | ||
|
|
||
| // Shared states (e.g. sessionHandlers) could be accessed by multiple threads (such as heartbeat thread), hence, | ||
| // it is necessary to acquire a lock on the fetcher instance before modifying the states. | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| synchronized (Fetcher.this) { | ||
| // we do not need to re-enable wakeups since we are closing already | ||
| client.disableWakeups(); | ||
| if (nextInLineFetch != null) | ||
| nextInLineFetch.drain(); | ||
| maybeCloseFetchSessions(timer); | ||
| Utils.closeQuietly(decompressionBufferSupplier, "decompressionBufferSupplier"); | ||
| sessionHandlers.clear(); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| if (nextInLineFetch != null) | ||
| nextInLineFetch.drain(); | ||
| decompressionBufferSupplier.close(); | ||
| close(time.timer(0)); | ||
| } | ||
|
|
||
| private Set<String> topicsForPartitions(Collection<TopicPartition> partitions) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -114,9 +114,18 @@ public boolean equals(Object o) { | |
| } | ||
|
|
||
| /** | ||
| * Return the metadata for the next error response. | ||
| * Return the metadata for the next request. The metadata is set to indicate that the client wants to close the | ||
| * existing session. | ||
| */ | ||
| public FetchMetadata nextCloseExisting() { | ||
| return new FetchMetadata(sessionId, FINAL_EPOCH); | ||
|
||
| } | ||
|
|
||
| /** | ||
| * Return the metadata for the next request. The metadata is set to indicate that the client wants to close the | ||
| * existing session and create a new one if possible. | ||
| */ | ||
| public FetchMetadata nextCloseExistingAttemptNew() { | ||
divijvaidya marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| return new FetchMetadata(sessionId, INITIAL_EPOCH); | ||
| } | ||
|
|
||
|
|
||
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.
What would happen if the session handler is reused after this is called? Should we add unit tests in
FetchSessionHandlerTestto be complete?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.
SessionHandlershould not be reused here after close because:Fetcheris only called from theConsumer.Consumerhas a single threaded access i.e. while it is processing theclose, we don't expect it to poll or callFetcher.sendFetches, session handler will not be used.SessionHandlermap will be cleared after the close request is sent in theFetcher.close()Fetcherwhile it is being closed by acquiring a lock onFetcher(atsynchronized (Fetcher.this)) before close starts. This ensures that sessionHandler is not called by anyone before close is complete (which should clear the sessionHandler map).Is my understanding correct here?
Regarding the test, what kind validation/assertion would you like to see from it? I can't think of a test that might be useful for us here.