This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 0ba6f3212d8 KAFKA-17903 Remove KafkaFuture#Function and 
KafkaFuture#thenApply (#17644)
0ba6f3212d8 is described below

commit 0ba6f3212d8ed16fa2aa5db0befab0459cb2d778
Author: Yung <[email protected]>
AuthorDate: Thu Oct 31 14:45:02 2024 +0800

    KAFKA-17903 Remove KafkaFuture#Function and KafkaFuture#thenApply (#17644)
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../main/java/org/apache/kafka/clients/admin/Admin.java    |  2 +-
 .../src/main/java/org/apache/kafka/common/KafkaFuture.java | 14 --------------
 .../org/apache/kafka/common/internals/KafkaFutureImpl.java | 10 ----------
 3 files changed, 1 insertion(+), 25 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index f5513d51f3b..6470042754c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -53,7 +53,7 @@ import java.util.Set;
  * Instances returned from the {@code create} methods of this interface are 
guaranteed to be thread safe.
  * However, the {@link KafkaFuture KafkaFutures} returned from request methods 
are executed
  * by a single thread so it is important that any code which executes on that 
thread when they complete
- * (using {@link KafkaFuture#thenApply(KafkaFuture.Function)}, for example) 
doesn't block
+ * (using {@link KafkaFuture#thenApply(KafkaFuture.BaseFunction)}, for 
example) doesn't block
  * for too long. If necessary, processing of results should be passed to 
another thread.
  * <p>
  * The operations exposed by Admin follow a consistent pattern:
diff --git a/clients/src/main/java/org/apache/kafka/common/KafkaFuture.java 
b/clients/src/main/java/org/apache/kafka/common/KafkaFuture.java
index a70afa8a3d7..d4b5f35314b 100644
--- a/clients/src/main/java/org/apache/kafka/common/KafkaFuture.java
+++ b/clients/src/main/java/org/apache/kafka/common/KafkaFuture.java
@@ -47,14 +47,6 @@ public abstract class KafkaFuture<T> implements Future<T> {
         B apply(A a);
     }
 
-    /**
-     * A function which takes objects of type A and returns objects of type B.
-     *
-     * @deprecated Since Kafka 3.0. Use the {@link BaseFunction} functional 
interface.
-     */
-    @Deprecated
-    public abstract static class Function<A, B> implements BaseFunction<A, B> 
{ }
-
     /**
      * A consumer of two different types of object.
      */
@@ -126,12 +118,6 @@ public abstract class KafkaFuture<T> implements Future<T> {
      */
     public abstract <R> KafkaFuture<R> thenApply(BaseFunction<T, R> function);
 
-    /**
-     * Prefer {@link KafkaFuture#thenApply(BaseFunction)} as this function is 
here for backwards compatibility reasons
-     * and might be deprecated/removed in a future release.
-     */
-    public abstract <R> KafkaFuture<R> thenApply(Function<T, R> function);
-
     /**
      * Returns a new KafkaFuture with the same result or exception as this 
future, that executes the given action
      * when this future completes.
diff --git 
a/clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java 
b/clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java
index 0d88c91740d..b2e93abaae7 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/internals/KafkaFutureImpl.java
@@ -90,16 +90,6 @@ public class KafkaFutureImpl<T> extends KafkaFuture<T> {
         }
     }
 
-    /**
-     * @see KafkaFutureImpl#thenApply(BaseFunction)
-     * @deprecated Since Kafka 3.0.
-     */
-    @Deprecated
-    @Override
-    public <R> KafkaFuture<R> thenApply(Function<T, R> function) {
-        return thenApply((BaseFunction<T, R>) function);
-    }
-
     @Override
     public KafkaFuture<T> whenComplete(final BiConsumer<? super T, ? super 
Throwable> biConsumer) {
         CompletableFuture<T> tCompletableFuture = 
completableFuture.whenComplete((java.util.function.BiConsumer<? super T, ? 
super Throwable>) (a, b) -> {

Reply via email to