You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "hachikuji (via GitHub)" <gi...@apache.org> on 2023/05/12 20:16:41 UTC

[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<CompletableFuture<?>, 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<K, V> 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<K, V> 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org