junrao commented on code in PR #14118: URL: https://github.com/apache/kafka/pull/14118#discussion_r1304862712
########## 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. Review Comment: taks => tasks Also, this method doesn't return anything so it's weird to have "return null here". ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java: ########## @@ -110,7 +110,7 @@ private boolean process(final OffsetFetchApplicationEvent event) { Optional<RequestManager> commitRequestManger = registry.get(RequestManager.Type.COMMIT); CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = event.future(); if (!commitRequestManger.isPresent()) { - future.completeExceptionally(new KafkaException("Unable to fetch committed offset because the " + + event.future().completeExceptionally(new KafkaException("Unable to fetch committed offset because the " + Review Comment: Hmm, why can't we just use `future`? ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java: ########## @@ -0,0 +1,77 @@ +/* + * 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 org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class WakeupTriggerTest { + private static long defaultTimeoutMs = 1000; + private WakeupTrigger wakeupTrigger; + + @BeforeEach + public void setup() { + this.wakeupTrigger = new WakeupTrigger(); + } + @Test Review Comment: Add a new line above. ########## 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. Review Comment: wakedup => wokenUp ########## 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 Review Comment: Indentation. Ditto below. ########## 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: Do we ever clear a WakedupFuture from pendingTask if no active task is set afterward? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -469,17 +474,19 @@ public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets) { @Override public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets, Duration timeout) { - CompletableFuture<Void> commitFuture = commit(offsets); + CompletableFuture<Void> commitFuture = wakeupTrigger.setActiveTask(commit(offsets)); Review Comment: Do we need to call `setActiveTask` before the event is added to eventHandler like we did in `committed()`? -- 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]
