KAFKA-17905 Remove the specified type of using lambda for BaseFunction (#17648)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Yung 2024-11-01 02:33:16 +08:00 committed by GitHub
parent dd432c0ca1
commit 6094882315
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 5 additions and 7 deletions

View File

@ -39,7 +39,7 @@ public class ListConsumerGroupsResult {
this.all = new KafkaFutureImpl<>();
this.valid = new KafkaFutureImpl<>();
this.errors = new KafkaFutureImpl<>();
future.thenApply((KafkaFuture.BaseFunction<Collection<Object>, Void>) results -> {
future.thenApply(results -> {
ArrayList<Throwable> curErrors = new ArrayList<>();
ArrayList<ConsumerGroupListing> curValid = new ArrayList<>();
for (Object resultObject : results) {

View File

@ -40,7 +40,7 @@ public class ListShareGroupsResult {
this.all = new KafkaFutureImpl<>();
this.valid = new KafkaFutureImpl<>();
this.errors = new KafkaFutureImpl<>();
future.thenApply((KafkaFuture.BaseFunction<Collection<Object>, Void>) results -> {
future.thenApply(results -> {
ArrayList<Throwable> curErrors = new ArrayList<>();
ArrayList<ShareGroupListing> curValid = new ArrayList<>();
for (Object resultObject : results) {

View File

@ -17,7 +17,6 @@
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import org.apache.kafka.common.utils.MockTime;
@ -298,7 +297,7 @@ public class RefreshingHttpsJwksTest extends OAuthBearerTest {
public <T> ScheduledFuture<T> schedule(final Callable<T> callable, long delayMs, Long period) {
KafkaFutureImpl<Long> waiter = new KafkaFutureImpl<>();
waiter.thenApply((KafkaFuture.BaseFunction<Long, Void>) now -> {
waiter.thenApply(now -> {
try {
callable.call();
} catch (Throwable e) {

View File

@ -16,7 +16,6 @@
*/
package org.apache.kafka.common.utils;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import org.slf4j.Logger;
@ -85,7 +84,7 @@ public class MockScheduler implements Scheduler, MockTime.Listener {
final Callable<T> callable, long delayMs) {
final KafkaFutureImpl<T> future = new KafkaFutureImpl<>();
KafkaFutureImpl<Long> waiter = new KafkaFutureImpl<>();
waiter.thenApply((KafkaFuture.BaseFunction<Long, Void>) now -> {
waiter.thenApply(now -> {
executor.submit((Callable<Void>) () -> {
// Note: it is possible that we'll execute Callable#call right after
// the future is cancelled. This is a valid sequence of events

View File

@ -333,7 +333,7 @@ public final class WorkerManager {
return worker.doneFuture;
}
KafkaFutureImpl<String> haltFuture = new KafkaFutureImpl<>();
haltFuture.thenApply((KafkaFuture.BaseFunction<String, Void>) errorString -> {
haltFuture.thenApply(errorString -> {
if (errorString == null)
errorString = "";
if (errorString.isEmpty()) {