You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 2 Next »

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state: Under Discussion

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

The admin client makes extensive use of KafkaFuture in its *Result classes to enable clients the choice between synchronous and asynchronous programming styles. KafkaFuture originally had to work on versions of Java which didn't support CompletableFuture, and so was intended to provide a useful subset of that missing JDK functionality. That constraint no longer applies. It would be beneficial to provide a richer API, like that of CompletableFuture, or at least CompletionStage, and also compatibility with 3rd party APIs which require an actual CompletableFuture or CompletionStage instance.

This KIP proposes to add a KafkaFuture#toCompletableFuture() method as a backward compatible solution to this problem.

toCompletableFuture() method is preferred over toCompletionStage because CompletableFuture implements CompletionStage anyway, so for callers who actually need a CompletableFuture it's slightly simpler to call toCompletableFuture() than toCompletionStage().toCompletableFuture() (and CompletableFuture#toCompletableFuture() returns this anyway, so there's nothing to be gained from trying to hide the CompletableFuture instance).

Now also seems like a good opportunity to:

  • Remove the @InterfaceStability.Evolving annotation on KafkaFuture to reflect the reality that changing this class incompatibly would cause of too much user code to break.
  • Deprecate the static class KafkaFuture.Function, which already had Javadoc noting that KafkaFuture.BaseFunction was preferred.

Future admin client methods would continue to use KafkaFuture for the sake of consistency.

Public Interfaces

The changes to KafkaFuture are summarized below:

/**
 * 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.
 * A flexible future which supports call chaining and other asynchronous programming patterns.
 *
 * <h3>Relation to {@code CompletableFuture}</h3>
 * <p>This class exists because support for a Future-like construct was once needed on Java versions predating
 * the addition of {@code CompletableFuture}. It is now possible to obtain a {@code CompletableFuture} from a
 * {@code KafkaFuture} instance by calling {@link #toCompletableFuture()}.
 * If converting {@link KafkaFuture#whenComplete(BiConsumer)} or {@link KafkaFuture#thenApply(BaseFunction)} to
 * {@link CompletableFuture#whenComplete(java.util.function.BiConsumer)} or
 * {@link CompletableFuture#thenApply(java.util.function.Function)} be aware that the returned
 * {@code KafkaFuture} will fail with an {@code ExecutionException}, where as a {@code CompletableFuture} fails
 * with a {@code CompletionException}.
 */
class KafkaFuture<Void> {

  // ... existing methods ...
  
  /**
   * Get a CompletableFuture with the same completion properties as this KafkaFuture.
   * The returned future will complete when this future completes and in the same way (with the same result or exception).
   */
  CompletableFuture toCompletableFuture();
  
  /**
   * A function which takes objects of type A and returns objects of type B.
   *
   * Prefer the functional interface {@link BaseFunction} over the class {@link Function}.  This class is here for
   * backwards compatibility reasons and might be deprecated/removed in a future release.
   * @deprecated Replaced by the functional interface {@link BaseFunction} over the class {@link Function}.
   */
  @Deprecated
  public static abstract class Function<A, B> implements BaseFunction<A, B> { }

}

Proposed Changes

Some work has already been done to thoroughly test the existing KafkaFuture API and reimplement it using a CompletableFuture. This KIP would allow access to the wrapped CompletableFuture instance.

KafkaFutureImpl would gain a new public constructor for wrapping a CompletableFuture, which will allow implementation of KafkaFuture#allOf() to be simplified.

Compatibility, Deprecation, and Migration Plan

The addition of toCompetableFuture is backwards compatible.

As noted, KafkaFuture.Function will be deprecated. The lambda-compatible BaseFunction has existed and been documented as preferred for since Kafka XXX.
The actual removal of KafkaFuture.Function can be done in some future major version of Kafka.

Rejected Alternatives

KafkaFuture implementing CompletionStage

KafkaFuture already has a thenApply(KafkaFuture.BaseFunction) method. Making KafkaFuture implement CompletionStage would require adding thenApply(java.util.function.Function). That is not a source compatible because existing call sites using lambdas would become ambiguous, since both parameter types are SAM types. While it's easily resolved with a type cast, it's still incompatible. There would also be differences in the exception handling for thenApply and whenComplete in order to keep KafkaFuture compatible with its current behaviour.

Specifically, the CompletionStage contract states:

In all other cases, if a stage's computation terminates abruptly
with an (unchecked) exception or error, then all dependent stages
requiring its completion complete exceptionally as well, with a
{@link CompletionException} holding the exception as its cause.


But KafkaFuture#thenApply(KafkaFuture.BaseFunction) always fails using an ExecutionException. So the two thenApply and whenComplete methods would behave differently and resolving the compiler error by casting a lambda to java.util.function.Function could break users' exception handling.

"Admin2"

Other, more radical, possibilities include deprecating and replacing KafkaFuture-returning methods on a case-by-case basis (eugh), or creating a new Admin2 client which used CompletionStage or CompletableFuture rather than KafkaFuture in its API, but was a thin wrapper of the existing Admin.
These have a high cost to existing users of the admin client, who would have to change their code. They also come at a greater cost in terms of additional testing and documentation overhead for the project. And they don't offer any extra functionality beyond the solution proposed.

It is possible that this cost/benefit analysis might change in the future, for example if Project Loom's virtual threading model
proves to be successful then having an Admin2 client which supported only a synchronous programming model could make sense.


  • No labels