mimaison commented on code in PR #19390: URL: https://github.com/apache/kafka/pull/19390#discussion_r2032754487
########## core/src/test/scala/integration/kafka/server/DelayedFutureTest.scala: ########## @@ -16,22 +16,22 @@ */ package integration.kafka.server -import kafka.server.DelayedFuturePurgatory import kafka.utils.TestUtils import org.apache.kafka.common.utils.Time +import org.apache.kafka.server.purgatory.DelayedFuturePurgatory import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.Test import java.util.concurrent.{CompletableFuture, ExecutionException} import java.util.concurrent.atomic.AtomicInteger -import scala.jdk.CollectionConverters.CollectionHasAsScala +import scala.jdk.CollectionConverters._ Review Comment: Can we keep the explicit import? ########## server/src/main/java/org/apache/kafka/server/purgatory/DelayedFuture.java: ########## @@ -0,0 +1,84 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; + +import org.slf4j.Logger; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * A delayed operation using CompletionFutures that can be created by KafkaApis and watched + * in a DelayedFuturePurgatory purgatory. This is used for ACL updates using async Authorizers. + */ +public class DelayedFuture<T> extends DelayedOperation { + + private final Logger log = new LogContext().logger(DelayedFuture.class.getName()); + private final List<CompletableFuture<T>> futures; + private final Runnable responseCallback; + private final long timeoutMs; + + public DelayedFuture(long timeoutMs, List<CompletableFuture<T>> futures, Runnable responseCallback) { + super(timeoutMs); + this.timeoutMs = timeoutMs; + this.futures = futures; + this.responseCallback = responseCallback; + } + + /** + * The operation can be completed if all the futures have completed successfully + * or failed with exceptions. + */ + @Override + public boolean tryComplete() { + log.trace("Trying to complete operation for {} futures", futures.size()); + + long pending = futures.stream().filter(future -> !future.isDone()).count(); + if (pending == 0) { + log.trace("All futures have been completed or have errors, completing the delayed operation"); + return forceComplete(); + } else { + log.trace(pending + " future still pending, not completing the delayed operation"); + return false; + } + } + + /** + * Timeout any pending futures and invoke responseCallback. This is invoked when all + * futures have completed or the operation has timed out. + */ + @Override + public void onComplete() { + List<CompletableFuture<T>> pendingFutures = futures.stream().filter(future -> !future.isDone()).collect(Collectors.toList()); Review Comment: You can use `toList()` instead of `collect(Collectors.toList())` ########## core/src/test/scala/integration/kafka/server/DelayedFutureTest.scala: ########## @@ -16,22 +16,22 @@ */ package integration.kafka.server Review Comment: I think we should also convert and move this file to server ########## server/src/main/java/org/apache/kafka/server/purgatory/DelayedFuturePurgatory.java: ########## @@ -0,0 +1,89 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.utils.KafkaThread; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; + +public class DelayedFuturePurgatory { + private final DelayedOperationPurgatory<DelayedFuture<?>> purgatory; + private final ThreadPoolExecutor executor; + private final DelayedOperationKey purgatoryKey; + + public DelayedFuturePurgatory(String purgatoryName, int brokerId) { + this.purgatory = new DelayedOperationPurgatory<>(purgatoryName, brokerId); + this.executor = new ThreadPoolExecutor( + 1, + 1, + 0, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new KafkaThread("DelayedExecutor-" + purgatoryName, r, true); + } + }); + this.purgatoryKey = new DelayedOperationKey() { Review Comment: ditto ########## server/src/main/java/org/apache/kafka/server/purgatory/DelayedFuturePurgatory.java: ########## @@ -0,0 +1,89 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.utils.KafkaThread; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; + +public class DelayedFuturePurgatory { + private final DelayedOperationPurgatory<DelayedFuture<?>> purgatory; + private final ThreadPoolExecutor executor; + private final DelayedOperationKey purgatoryKey; + + public DelayedFuturePurgatory(String purgatoryName, int brokerId) { + this.purgatory = new DelayedOperationPurgatory<>(purgatoryName, brokerId); + this.executor = new ThreadPoolExecutor( + 1, + 1, + 0, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new KafkaThread("DelayedExecutor-" + purgatoryName, r, true); + } + }); + this.purgatoryKey = new DelayedOperationKey() { + @Override + public String keyLabel() { + return "delayed-future-key"; + } + }; + } + + public <T> DelayedFuture<T> tryCompleteElseWatch( + long timeoutMs, + List<CompletableFuture<T>> futures, + Runnable responseCallback + ) { + DelayedFuture<T> delayedFuture = new DelayedFuture<>(timeoutMs, futures, responseCallback); + boolean done = purgatory.tryCompleteElseWatch(delayedFuture, List.of(purgatoryKey)); + if (!done) { + BiConsumer<Void, Throwable> callbackAction = new BiConsumer<>() { Review Comment: ditto ########## server/src/main/java/org/apache/kafka/server/purgatory/DelayedFuture.java: ########## @@ -0,0 +1,84 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; + +import org.slf4j.Logger; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * A delayed operation using CompletionFutures that can be created by KafkaApis and watched + * in a DelayedFuturePurgatory purgatory. This is used for ACL updates using async Authorizers. + */ +public class DelayedFuture<T> extends DelayedOperation { + + private final Logger log = new LogContext().logger(DelayedFuture.class.getName()); + private final List<CompletableFuture<T>> futures; + private final Runnable responseCallback; + private final long timeoutMs; + + public DelayedFuture(long timeoutMs, List<CompletableFuture<T>> futures, Runnable responseCallback) { + super(timeoutMs); + this.timeoutMs = timeoutMs; + this.futures = futures; + this.responseCallback = responseCallback; + } + + /** + * The operation can be completed if all the futures have completed successfully + * or failed with exceptions. + */ + @Override + public boolean tryComplete() { + log.trace("Trying to complete operation for {} futures", futures.size()); + + long pending = futures.stream().filter(future -> !future.isDone()).count(); + if (pending == 0) { + log.trace("All futures have been completed or have errors, completing the delayed operation"); + return forceComplete(); + } else { + log.trace(pending + " future still pending, not completing the delayed operation"); Review Comment: Use templating `{}` instead of concatenation in Logger methods. ########## server/src/main/java/org/apache/kafka/server/purgatory/DelayedFuturePurgatory.java: ########## @@ -0,0 +1,89 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.utils.KafkaThread; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; + +public class DelayedFuturePurgatory { + private final DelayedOperationPurgatory<DelayedFuture<?>> purgatory; + private final ThreadPoolExecutor executor; + private final DelayedOperationKey purgatoryKey; + + public DelayedFuturePurgatory(String purgatoryName, int brokerId) { + this.purgatory = new DelayedOperationPurgatory<>(purgatoryName, brokerId); + this.executor = new ThreadPoolExecutor( + 1, + 1, + 0, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactory() { Review Comment: You can use a lambda to make the anonymous class a bit less verbose -- 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