philipnee commented on code in PR #14118: URL: https://github.com/apache/kafka/pull/14118#discussion_r1308022177
########## 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: hey @junrao - sorry I misunderstood. In the example you provided, in fact, that is the current behavior. It can be replicated in the integration test like this: ``` @Test def testWakeup(): Unit = { val consumer = createConsumer() val producer = createProducer() val numRecords = 10000 val startingTimestamp = System.currentTimeMillis() sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) consumer.assign(List(tp).asJava) consumer.wakeup(); // wakeup w/o pending task assertThrows(classOf[WakeupException], () => consumer.poll(Duration.of(1000, java.time.temporal .ChronoUnit .MILLIS))) // throws immediately println(consumer.poll(Duration.of(1000, java.time.temporal.ChronoUnit.MILLIS))) // prints some records } ``` In short - consumer.poll shouldn't clear WakeupFuture. In the Javadoc, it also didn't make an exception for the poll call see: `...the next call to such a method will raise it instead.` ``` Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll. The thread which is blocking in an operation will throw [WakeupException](https://kafka.apache.org/31/javadoc/org/apache/kafka/common/errors/WakeupException.html). If no thread is blocking in a method which can throw [WakeupException](https://kafka.apache.org/31/javadoc/org/apache/kafka/common/errors/WakeupException.html), the next call to such a method will raise it instead. ``` -- 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]
