junrao commented on code in PR #14118:
URL: https://github.com/apache/kafka/pull/14118#discussion_r1306021925


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.errors.WakeupException;
+
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Ensures blocking APIs can be woken up by the consumer.wakeup().
+ */
+public class WakeupTrigger {
+    private AtomicReference<Wakeupable> pendingTask = new 
AtomicReference<>(null);
+
+    /*
+      Wakeup a pending task.  If there isn't any pending task, return a 
WakedupFuture, so that the subsequent call
+      would know wakeup was previously called.
+
+      If there are active tasks, complete it with WakeupException, then unset 
pending task (return null here.
+      If the current task has already been woken-up, do nothing.
+     */
+    public void wakeup() {
+        pendingTask.getAndUpdate(task -> {
+            if (task == null) {
+                return new WakedupFuture();
+            } else if (task instanceof ActiveFuture) {
+                ActiveFuture active = (ActiveFuture) task;
+                active.future().completeExceptionally(new WakeupException());
+                return null;
+            } else {
+                return task;
+            }
+        });
+    }
+
+    /*
+    If there is no pending task, set the pending task active.
+    If wakeup was called before setting an active task, the current task will 
complete exceptionally with
+    WakeupException right
+    away.
+    if there is an active task, throw exception.
+     */
+    public <T> CompletableFuture<T> setActiveTask(final CompletableFuture<T> 
currentTask) {
+        Objects.requireNonNull(currentTask, "currentTask cannot be null");
+        pendingTask.getAndUpdate(task -> {
+            if (task == null) {
+                return new ActiveFuture(currentTask);
+            } else if (task instanceof WakedupFuture) {
+                currentTask.completeExceptionally(new WakeupException());
+                return null;
+            }
+            // last active state is still active
+            throw new KafkaException("Last active task is still active");
+        });
+        return currentTask;
+    }
+
+    public void clearActiveTask() {
+        pendingTask.getAndUpdate(task -> {
+            if (task == null) {
+                return null;
+            } else if (task instanceof ActiveFuture) {
+                return null;
+            }
+            return task;
+        });
+    }
+
+    Wakeupable getPendingTask() {
+        return pendingTask.get();
+    }
+
+    interface Wakeupable { }
+
+    static class ActiveFuture implements Wakeupable {
+        private final CompletableFuture<?> future;
+
+        public ActiveFuture(final CompletableFuture<?> future) {
+            this.future = future;
+        }
+
+        public CompletableFuture<?> future() {
+            return future;
+        }
+    }
+
+    static class WakedupFuture implements Wakeupable { }

Review Comment:
   Should this be WokenUpFuture?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.errors.WakeupException;
+
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Ensures blocking APIs can be woken up by the consumer.wakeup().
+ */
+public class WakeupTrigger {
+    private AtomicReference<Wakeupable> pendingTask = new 
AtomicReference<>(null);
+
+    /*
+    Wakeup a pending task.  If there isn't any pending task, return a 
WakedupFuture, so that the subsequent call
+    would know wakeup was previously called.
+
+    If there are active taks, complete it with WakeupException, then unset 
pending task (return null here.
+    If the current task has already been wakedup, do nothing.
+     */
+    public void wakeup() {
+        pendingTask.getAndUpdate(task -> {
+            if (task == null) {
+                return new WakedupFuture();

Review Comment:
   I was wondering when an application calls `Consumer.wakeup`. If it only 
calls when it intends to terminate, there is no need to clear WakedupFuture.
   
   However, it seems that an application can call `Consumer.wakeup` to unblock 
a long `poll` call and then continue to make new `poll` calls. If we don't 
clear `WakedupFuture`, future blocking calls after the new `poll` call could 
fail unexpectedly?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to