tombentley commented on a change in pull request #9878: URL: https://github.com/apache/kafka/pull/9878#discussion_r663752257
########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaCompletableFuture.java ########## @@ -0,0 +1,79 @@ +/* + * 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.common.internals; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * This internal class exists because CompletableFuture exposes complete(), completeExceptionally() and + * other methods which would allow erroneous completion by user code of a KafkaFuture returned from a + * Kafka API to a client application. + * @param <T> The type of the future value. + */ +public class KafkaCompletableFuture<T> extends CompletableFuture<T> { Review comment: The KIP describes it like this: > Adding toCompletionStage() is sufficient because CompletionStage itself exposes toCompletableFuture(), so anyone who needs an actual CompletableFuture (e.g. for interoperating with 3rd party APIs that require one) can get one. However, CompletableFuture exposes methods for future completion which should not be called by users (only the Admin client should be completing the returned futures), so calling these will be prevented. It is expected that users wanting to block on the completion of the KafkaFuture would use kafkaFuture.get(), rather than calling kafkaFuture.toCompletionStage().toCompletableFuture().get(), so the need to access the CompleteableFuture should be rare. Essentially we want to allow access to a `CompletableFuture` to allow interop with 3rd part APIs which might require it (e.g. for using the blocking `get` methods) , but not allow user code to complete those futures. The `CompletableFuture` API unhelpfully adds _both_ the blocking `get` methods and the `complete` methods. ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaCompletableFuture.java ########## @@ -0,0 +1,79 @@ +/* + * 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.common.internals; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * This internal class exists because CompletableFuture exposes complete(), completeExceptionally() and + * other methods which would allow erroneous completion by user code of a KafkaFuture returned from a + * Kafka API to a client application. + * @param <T> The type of the future value. + */ +public class KafkaCompletableFuture<T> extends CompletableFuture<T> { + + boolean kafkaComplete(T value) { Review comment: The idea is that all Kafka code knows it's using a `KafkaCompletableFuture` and knows to complete it via the `kafka*()` methods. User code which erroneously tries to complete via `complete()` gets the exception thrown. ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java ########## @@ -267,50 +180,82 @@ public T get() throws InterruptedException, ExecutionException { @Override public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - SingleWaiter<T> waiter = new SingleWaiter<>(); - addWaiter(waiter); - return waiter.await(timeout, unit); + try { + return completableFuture.get(timeout, unit); + } catch (ExecutionException e) { + maybeThrowCancellationException(e.getCause()); + throw e; + } } /** * Returns the result value (or throws any encountered exception) if completed, else returns * the given valueIfAbsent. */ @Override - public synchronized T getNow(T valueIfAbsent) throws InterruptedException, ExecutionException { - if (exception != null) - wrapAndThrow(exception); - if (done) - return value; - return valueIfAbsent; + public synchronized T getNow(T valueIfAbsent) throws ExecutionException { Review comment: > Does it need to remove InterruptedException from other get methods also? `get()` and `get(long, TimeUnit)` can still throw `InterruptedException` due to the blocking behaviour (the `completableFuture.get` that we delegate to throws `InterruptedException`). > For another, why we don't remove InterruptedException from KafkaFuture#getNow? Good spot! `KafkaFuture#getNow` should never have declared that and it's annoying for callers to have to deal with it. But this wasn't described in the KIP and wouldn't be a source compatible change (existing code with a `catch (InterruptedException)` would get a compile error). I'd be happy to remove it and notify the VOTE thread, WDYT @kkonstantine @dajac ? ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaCompletableFuture.java ########## @@ -0,0 +1,79 @@ +/* + * 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.common.internals; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * This internal class exists because CompletableFuture exposes complete(), completeExceptionally() and + * other methods which would allow erroneous completion by user code of a KafkaFuture returned from a + * Kafka API to a client application. + * @param <T> The type of the future value. + */ +public class KafkaCompletableFuture<T> extends CompletableFuture<T> { + + boolean kafkaComplete(T value) { Review comment: Done ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java ########## @@ -267,50 +180,82 @@ public T get() throws InterruptedException, ExecutionException { @Override public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - SingleWaiter<T> waiter = new SingleWaiter<>(); - addWaiter(waiter); - return waiter.await(timeout, unit); + try { + return completableFuture.get(timeout, unit); + } catch (ExecutionException e) { + maybeThrowCancellationException(e.getCause()); + throw e; + } } /** * Returns the result value (or throws any encountered exception) if completed, else returns * the given valueIfAbsent. */ @Override - public synchronized T getNow(T valueIfAbsent) throws InterruptedException, ExecutionException { - if (exception != null) - wrapAndThrow(exception); - if (done) - return value; - return valueIfAbsent; + public synchronized T getNow(T valueIfAbsent) throws ExecutionException { Review comment: I wouldn't call it `get` because it has different semantics to the existing get methods, so I think it should have a distinct name. And I can't think of a succinct alternative to `getNow`. Something like `getOrDefault` doesn't convey they key difference from `get` (the lack of blocking). `getImmediately` is just a bit verbose. The API of KafkaFuture has always been a subset of `CompletableFuture`. Introducing a new method would break that. While it would be discoverable while the deprecated method remained to point the way it wouldn't be as discoverable once the old `getNow` was removed. All deprecation really does is let the programmer defer fixing the problem. But it's most likely not a huge problem to fix (i.e. they've likely caught the exception close to the method call, rather than defining many layers of API with `throws InterruptedException`). A new method also means their code is then not binary compatible with older client versions. So on balance I personally favour just removing the `throws InterruptedException` in 3.0 and letting the programmer clean up the code sooner rather than later. There is always the "do nothing" alternative. ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java ########## @@ -27,217 +28,111 @@ /** * A flexible future which supports call chaining and other asynchronous programming patterns. - * This will eventually become a thin shim on top of Java 8's CompletableFuture. */ public class KafkaFutureImpl<T> extends KafkaFuture<T> { - /** - * A convenience method that throws the current exception, wrapping it if needed. - * - * In general, KafkaFuture throws CancellationException and InterruptedException directly, and - * wraps all other exceptions in an ExecutionException. - */ - private static void wrapAndThrow(Throwable t) throws InterruptedException, ExecutionException { - if (t instanceof CancellationException) { - throw (CancellationException) t; - } else if (t instanceof InterruptedException) { - throw (InterruptedException) t; - } else { - throw new ExecutionException(t); - } - } - private static class Applicant<A, B> implements BiConsumer<A, Throwable> { - private final BaseFunction<A, B> function; - private final KafkaFutureImpl<B> future; + private final KafkaCompletableFuture<T> completableFuture; - Applicant(BaseFunction<A, B> function, KafkaFutureImpl<B> future) { - this.function = function; - this.future = future; - } + private final boolean isDependant; - @Override - public void accept(A a, Throwable exception) { - if (exception != null) { - future.completeExceptionally(exception); - } else { - try { - B b = function.apply(a); - future.complete(b); - } catch (Throwable t) { - future.completeExceptionally(t); - } - } - } + public KafkaFutureImpl() { + this(false, new KafkaCompletableFuture<>()); } - private static class SingleWaiter<R> implements BiConsumer<R, Throwable> { - private R value = null; - private Throwable exception = null; - private boolean done = false; - - @Override - public synchronized void accept(R newValue, Throwable newException) { - this.value = newValue; - this.exception = newException; - this.done = true; - this.notifyAll(); - } - - synchronized R await() throws InterruptedException, ExecutionException { - while (true) { - if (exception != null) - wrapAndThrow(exception); - if (done) - return value; - this.wait(); - } - } - - R await(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - long startMs = System.currentTimeMillis(); - long waitTimeMs = unit.toMillis(timeout); - long delta = 0; - synchronized (this) { - while (true) { - if (exception != null) - wrapAndThrow(exception); - if (done) - return value; - if (delta >= waitTimeMs) { - throw new TimeoutException(); - } - this.wait(waitTimeMs - delta); - delta = System.currentTimeMillis() - startMs; - } - } - } + public KafkaFutureImpl(KafkaCompletableFuture<T> completableFuture) { Review comment: No, but it makes me wonder whether we'd find factory methods for already completed and already failed `KafkaFutures` useful. ########## File path: clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java ########## @@ -267,50 +180,82 @@ public T get() throws InterruptedException, ExecutionException { @Override public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - SingleWaiter<T> waiter = new SingleWaiter<>(); - addWaiter(waiter); - return waiter.await(timeout, unit); + try { + return completableFuture.get(timeout, unit); + } catch (ExecutionException e) { + maybeThrowCancellationException(e.getCause()); + throw e; + } } /** * Returns the result value (or throws any encountered exception) if completed, else returns * the given valueIfAbsent. */ @Override - public synchronized T getNow(T valueIfAbsent) throws InterruptedException, ExecutionException { - if (exception != null) - wrapAndThrow(exception); - if (done) - return value; - return valueIfAbsent; + public synchronized T getNow(T valueIfAbsent) throws ExecutionException { Review comment: OK, I've reverted it. -- 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