[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1192841324 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -461,6 +507,10 @@ public void close(Duration timeout) { @Override public void wakeup() { +this.shouldWakeup.set(true); +if (this.activeFutures.get() != null) { +this.activeFutures.getAndSet(null).completeExceptionally(new WakeupException()); Review Comment: This does not seem safe. If there are concurrent calls to `getAndSet`, we might see NPE? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1192840218 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -319,27 +324,53 @@ public Map committed(final Set + * If the timeout specified by {@code default.api.timeout.ms} expires + * {@link org.apache.kafka.common.errors.TimeoutException} is thrown. + * + * @param partitions The partition to check + * @param timeout The maximum time to block. + * @return The last committed offset and metadata or null if there was no prior commit + * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this + * function is called + * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while + * this function is called + * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details + * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the + * configured groupId. See the exception for more details + * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors + * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before + * the timeout specified by {@code default.api.timeout.ms} expires. + */ @Override public Map committed(final Set partitions, final Duration timeout) { +maybeWakeup(); maybeThrowInvalidGroupIdException(); + if (partitions.isEmpty()) { return new HashMap<>(); } final OffsetFetchApplicationEvent event = new OffsetFetchApplicationEvent(partitions); +activeFutures.set(event.future()); eventHandler.add(event); try { -return event.complete(Duration.ofMillis(100)); +return event.complete(timeout); +} catch (ExecutionException e) { +throw new KafkaException(e); } catch (InterruptedException e) { throw new InterruptException(e); } catch (TimeoutException e) { throw new org.apache.kafka.common.errors.TimeoutException(e); -} catch (ExecutionException e) { -// Execution exception is thrown here -throw new KafkaException(e); -} catch (Exception e) { +} catch (WakeupException e) { Review Comment: Hmm, wouldn't we need to unwrap this exception from `ExecutionException`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1192838927 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -98,8 +97,7 @@ private final Metrics metrics; private final long defaultApiTimeoutMs; -private final ConcurrentHashMap.KeySetView, Boolean> activeFutures = -ConcurrentHashMap.newKeySet(); +private final AtomicReference activeFutures = new AtomicReference<>(); Review Comment: nit: `activeFuture`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1192768691 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -546,6 +619,22 @@ public ConsumerRecords poll(long timeout) { throw new KafkaException("method not implemented"); } +private void maybeWakeup() { +if (this.closed) +throw new IllegalStateException("This consumer has already been closed."); + +log.debug("Raising WakeupException in response to user wakeup"); Review Comment: Ok. Not a big deal either way. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1192760639 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -96,6 +98,11 @@ private final Metrics metrics; private final long defaultApiTimeoutMs; +private final ConcurrentHashMap.KeySetView, Boolean> activeFutures = Review Comment: Is there ever a case where we have multiple futures waited on? In other words, could this be a simpler type, such as `AtomicReference`? ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -546,6 +619,22 @@ public ConsumerRecords poll(long timeout) { throw new KafkaException("method not implemented"); } +private void maybeWakeup() { +if (this.closed) +throw new IllegalStateException("This consumer has already been closed."); + +log.debug("Raising WakeupException in response to user wakeup"); Review Comment: Not sure this is super useful. Maybe we could add the wakeup exception so that at least we get the stack trace? ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -546,6 +619,22 @@ public ConsumerRecords poll(long timeout) { throw new KafkaException("method not implemented"); } +private void maybeWakeup() { +if (this.closed) +throw new IllegalStateException("This consumer has already been closed."); Review Comment: Original consumer doesn't do this. Is there a good reason to change the behavior? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] hachikuji commented on a diff in pull request #13490: KAFKA-14875: Implement wakeup
hachikuji commented on code in PR #13490: URL: https://github.com/apache/kafka/pull/13490#discussion_r1169316955 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -250,6 +257,7 @@ public void commitAsync(OffsetCommitCallback callback) { @Override public void commitAsync(Map offsets, OffsetCommitCallback callback) { +maybeWakeup(); Review Comment: Why are we adding this? I don't think wakeup is expected to be thrown from async APIs? ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupableFuture.java: ## @@ -0,0 +1,33 @@ +/* + * 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.common.errors.WakeupException; + +import java.util.concurrent.CompletableFuture; + +/** + * Can be interrupted by calling {@link #wakeup()}. + */ +public class WakeupableFuture extends CompletableFuture { Review Comment: This class feels like overkill. The direct call to `completeExceptionally` seems clear already. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org