Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Added timeouts on send and flush calls in KafkaProducerWrapper #696

Open
wants to merge 6 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions config/server.properties
Original file line number Diff line number Diff line change
Expand Up @@ -31,3 +31,7 @@ brooklin.server.connector.test.strategy.TasksPerDatastream = 4

brooklin.server.connector.kafkaMirroringConnector.factoryClassName=com.linkedin.datastream.connectors.kafka.mirrormaker.KafkaMirrorMakerConnectorFactory
brooklin.server.connector.kafkaMirroringConnector.assignmentStrategyFactory=com.linkedin.datastream.server.assignment.BroadcastStrategyFactory

########################### Kafka Producer Wrapper Configs ####################
brooklin.server.kafkaProducerWrapper.sendTimeout=50000
brooklin.server.kafkaProducerWrapper.flushTimeout=3600000
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This wouldn't be the right way to pass configs to your KafkaProducerWrapper. Lets discuss how to do this offline

Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/**
* Copyright 2020 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information.
* See the NOTICE file in the project root for additional information regarding copyright ownership.
*/
package com.linkedin.datastream.kafka;

import java.time.Duration;
import java.util.Properties;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;

import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.errors.InterruptException;

import com.linkedin.datastream.common.CompletableFutureUtils;
import com.linkedin.datastream.common.VerifiableProperties;

/**
* An extension of {@link KafkaProducerWrapper} with bounded calls for flush and send
*/
class BoundedKafkaProducerWrapper<K, V> extends KafkaProducerWrapper<K, V> {
Copy link
Contributor

@ahmedahamid ahmedahamid Apr 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bounded is a little vague cause it's easy to confuse with buffering. I realize a better name won't be easy. If you can't think of one, just make sure the Javadoc is unambiguous (e.g. with timeouts for flush and send).

private static final int DEFAULT_SEND_TIME_OUT = 5000;
private static final int DEFAULT_FLUSH_TIME_OUT = 10 * 60 * 1000;
Copy link
Contributor

@ahmedahamid ahmedahamid Apr 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. Please encode time units into variable/config names, e.g. DEFAULT_SEND_TIME_OUT_MS
  2. Use longs for timeout configs (e.g. see existing timeout configs)
  3. If you like, you can use Duration methods to initialize (e.g. Duration.ofSeconds(5).toMillis())


private static final String SEND_TIMEOUT_CONFIG_KEY = "brooklin.server.kafkaProducerWrapper.sendTimeout";
private static final String FLUSH_TIMEOUT_CONFIG_KEY = "brooklin.server.kafkaProducerWrapper.flushTimeout";
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't the right way to pass configs. Any configs you want to pass here need to be scoped under the transport provider configs. The KafkaProducerWrapper receives all of the transport provider configs, so you'd want to add these under that scope. We don't need to access the full property name, because as the configs pass through the layers, the relevant prefixes are removed. You can see how other configs are accessed in KafkaProducerWrapper, you'll see they don't have that whole "brooklin.server" prefix. You can access these the same way.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@somandal I've discussed this with Ahmed. Will push the fixe soon


private int _sendTimeout;
private int _flushTimeout;
Copy link
Contributor

@ahmedahamid ahmedahamid Apr 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. final
  2. Add time unit suffixes (e.g. _sendTimoutMs)


BoundedKafkaProducerWrapper(String logSuffix, Properties props, String metricsNamesPrefix) {
super(logSuffix, props, metricsNamesPrefix);

VerifiableProperties properties = new VerifiableProperties(props);
_sendTimeout = properties.getInt(SEND_TIMEOUT_CONFIG_KEY, DEFAULT_SEND_TIME_OUT);
_flushTimeout = properties.getInt(FLUSH_TIMEOUT_CONFIG_KEY, DEFAULT_FLUSH_TIME_OUT);
}

@Override
void doSend(Producer<K, V> producer, ProducerRecord<K, V> record, Callback callback) {
CompletableFutureUtils.within(produceMessage(producer, record), Duration.ofMillis(_sendTimeout))
.thenAccept(m -> callback.onCompletion(m, null))
.exceptionally(completionEx -> {
Throwable cause = completionEx.getCause();
if (cause instanceof KafkaClientException) {
KafkaClientException ex = (KafkaClientException) cause;
callback.onCompletion(ex.getMetadata(), (Exception) ex.getCause());
} else if (cause instanceof java.util.concurrent.TimeoutException) {
_log.warn("KafkaProducerWrapper send timed out. The destination topic may be unavailable.");
callback.onCompletion(null, (java.util.concurrent.TimeoutException) cause);
}
return null;
});
}

private CompletableFuture<RecordMetadata> produceMessage(Producer<K, V> producer, ProducerRecord<K, V> record) {
CompletableFuture<RecordMetadata> future = new CompletableFuture<>();

producer.send(record, (metadata, exception) -> {
if (exception == null) {
future.complete(metadata);
} else {
future.completeExceptionally(new KafkaClientException(metadata, exception));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we need to propagate metadata if exception is null. The javadocs on Callback.onCompletion() state that metadata is null if an error occurred. This would also spare us having to introduce KafkaClientException.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There were tons of checks up the callback chain that are dealing with the metadata and exception. You're probably right, I need to see if it's safe to do so and remove KafkaClientException. I was kind of forced to introduce it in the first place.

}
});

return future;
}
Copy link
Contributor

@ahmedahamid ahmedahamid Apr 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking out loud:

  • This method is doing almost everything we need: it creates a CompletableFuture that is completed if the callback is called (send success/failure).
  • The only missing bit is canceling future after timeout elapses if future.isDone() is false, which can be accomplished with a ScheduledExecutorService. I know this is exactly what CompletableFutureUtils.failAfter() is doing but I think the logic over there is more than what's absolutely necessary; we don't really need the other CompletableFuture failAfter() creates or the additional logic in within(). We can just cancel this same future if it isn't done when timeout elapses.
     scheduler.schedule(() -> {
         if (!future.isDone()) {
             future.cancel();
         }
     }, _sendTimeout, TimeUnit.MILLISECONDS);
  • future.cancel() causes a java.util.concurrent.CancellationException to be thrown, which means we don't have to construct a TimeoutException ourselves because a CancellationException can only mean we cancelled it after the timeout elapsed.

This seems like something this method can do with a private executor service. I am not sure we really need a utils class just for this purpose.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ahmedahamid I just think that cancellation and timeout are semantically different. We may want to cancel the future after timeout in our case, but that's not necessarily true in general. Also, smth may be cancelled without waiting for timeout (based on user input or other external factors). Just thinking out loud. Will see whether I can get rid of the utils.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just think that cancellation and timeout are semantically different.

I didn't mean to suggest we should propagate CancellationException to callback. I was assuming doSend() will still construct a TimeoutException in case future is cancelled. That does sound a bit roundabout though; it would certainly be better to do future.completeExceptionally(new TimeoutException(...)) instead of future.cancel() if future.isDone() is false after timeout.


@Override
synchronized void flush() {
if (_kafkaProducer != null) {
try {
CompletableFutureUtils.within(CompletableFuture.runAsync(() -> _kafkaProducer.flush()),
Copy link
Contributor

@ahmedahamid ahmedahamid Apr 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CompletableFuture.runAsync() uses the common pool which isn't the best option for a potentially long blocking call like Producer.flush(). Even if we provide our own thread pool to runAsync(), the CompletableFuture we'll get won't give us a way to interrupt a Producer.flush() call that exceeds the allowed timeout, which is necessary to free up the thread-pool thread in question. This is because calling cancel(true) on a CompletableFuture returned by runAsync() only causes a cancellation exception to be propagated without interrupting the blocked thread pool.

I'm afraid our only option here seems to be using an ExecutorService directly

    // It's okay to use a single thread executor since flush() is synchronized
    ExecutorService executor = Executors.newSingleThreadExecutor();

    Future<?> future = executor.submit(() -> super.flush());
    try {
        // Block until timeout elapses
        future.get(_flushTimeout, TimeUnit.MILLISECONDS);
    } catch (TimeoutException e) {
        ...
        // Interrupt the Producer.flush() call to free up the blocked thread
        future.cancel(true);
        ...
    }

Duration.ofMillis(_flushTimeout)).join();
} catch (CompletionException e) {
Throwable cause = e.getCause();

if (cause instanceof InterruptException) {
// The KafkaProducer object should not be reused on an interrupted flush
_log.warn("Kafka producer flush interrupted, closing producer {}.", _kafkaProducer);
shutdownProducer();
throw (InterruptException) cause;
} else if (cause instanceof java.util.concurrent.TimeoutException) {
_log.warn("Kafka producer flush timed out after {}ms. Destination topic may be unavailable.", _flushTimeout);
}

throw e;
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/**
* Copyright 2020 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information.
* See the NOTICE file in the project root for additional information regarding copyright ownership.
*/

package com.linkedin.datastream.kafka;
import org.apache.kafka.clients.producer.RecordMetadata;

class KafkaClientException extends Exception {
private static final long serialVersionUID = 1;

private final RecordMetadata _metadata;

public RecordMetadata getMetadata() {
return _metadata;
}

public KafkaClientException(RecordMetadata metadata, Throwable innerException) {
super(innerException);
_metadata = metadata;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;

import org.apache.commons.lang.exception.ExceptionUtils;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nothing wrong with this but I'd recommend lang3; it generally offers more modern facilities (better support for more recent versions of Java) and it's the one we have an explicit dependency on.

This would also entail adding an explicit dependency for this module on it in build.gradle :

project(':datastream-kafka') {
  dependencies {
    ...
    compile "org.apache.commons:commons-lang3:$commonslang3Version"

import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
Expand Down Expand Up @@ -60,7 +61,8 @@ class KafkaProducerWrapper<K, V> {

private static final int TIME_OUT = 2000;
private static final int MAX_SEND_ATTEMPTS = 10;
private final Logger _log;

protected final Logger _log;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this with other protected member variables

private final long _sendFailureRetryWaitTimeMs;

private final String _clientId;
Expand All @@ -72,8 +74,7 @@ class KafkaProducerWrapper<K, V> {
// Producer is lazily initialized during the first send call.
// Also, can be nullified in case of exceptions, and recreated by subsequent send calls.
// Mark as volatile as it is mutable and used by different threads
private volatile Producer<K, V> _kafkaProducer;

protected volatile Producer<K, V> _kafkaProducer;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not necessary to mark member fields/methods protected if the extenders of this class live in the same package. This is only useful if you want to make them accessible to extenders in different packages.

Since KafkaProducerWrapper and BoundedKafkaProducerWrapper both live in the same package (com.linkedin.datastream.kafka), all package-private (no modifier) fields/methods in the former are accessible/overridable to/by the latter.

private final KafkaProducerFactory<K, V> _producerFactory;

// Limiter to control how fast producers are re-created after failures.
Expand Down Expand Up @@ -142,7 +143,7 @@ private void populateDefaultProducerConfigs() {
DEFAULT_MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_VALUE);
}

private Optional<Producer<K, V>> maybeGetKafkaProducer(DatastreamTask task) {
protected Optional<Producer<K, V>> maybeGetKafkaProducer(DatastreamTask task) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see you overriding this in your Bounded implementation, why make this protected?

Producer<K, V> producer = _kafkaProducer;
if (producer == null) {
producer = initializeProducer(task);
Expand Down Expand Up @@ -186,56 +187,62 @@ Producer<K, V> createKafkaProducer() {
return _producerFactory.createProducer(_props);
}

/**
* There are two known cases that lead to IllegalStateException and we should retry:
* (1) number of brokers is less than minISR
* (2) producer is closed in generateSendFailure by another thread
* (3) For either condition, we should retry as broker comes back healthy or producer is recreated
*/
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any reason why you moved these comments out? Also, I see that you've made the last line into (3), whereas it just talks about the above two points?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

To Sonam's first point: I think those comments are closely related to the logic within the method since they discuss impl details like exception types.

void send(DatastreamTask task, ProducerRecord<K, V> producerRecord, Callback onComplete)
throws InterruptedException {
// There are two known cases that lead to IllegalStateException and we should retry:
// 1) number of brokers is less than minISR
// 2) producer is closed in generateSendFailure by another thread
// For either condition, we should retry as broker comes back healthy or producer is recreated
boolean retry = true;
int numberOfAttempt = 0;
int numberOfAttempts = 0;

while (retry) {
try {
++numberOfAttempt;
maybeGetKafkaProducer(task).ifPresent(p -> p.send(producerRecord, (metadata, exception) -> {
if (exception == null) {
onComplete.onCompletion(metadata, null);
} else {
onComplete.onCompletion(metadata, generateSendFailure(exception));
}
}));
numberOfAttempts++;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change this back to ++numberOfAttempts?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

I like the rename (added s) but prefer the prefix form.

In fact, it wouldn't be such a bad idea to turn that loop into a for loop:

for (int numberOfAttempts = 1; retry; ++numberOfAttempts) {
  ...
}

or even eliminate retry entirely:

for (int numberOfAttempts = 1;; ++numberOfAttempts) {
    try {
        maybeGetKafkaProducer(task).ifPresent(p -> doSend(p, producerRecord, onComplete));
        return;
     } catch (...) {
         ...
     }
}


maybeGetKafkaProducer(task).ifPresent(p -> doSend(p, producerRecord, onComplete));

retry = false;
} catch (IllegalStateException e) {
//The following exception should be quite rare as most exceptions will be throw async callback
_log.warn("Either send is called on a closed producer or broker count is less than minISR, retry in {} ms.",
_sendFailureRetryWaitTimeMs, e);
Thread.sleep(_sendFailureRetryWaitTimeMs);
} catch (TimeoutException e) {
_log.warn("Kafka producer buffer is full, retry in {} ms.", _sendFailureRetryWaitTimeMs, e);
} catch (TimeoutException ex) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Was there some reason why you had to reorder the catches here? In the previous code, we catch IllegalStateException, and then catch TimeoutException, and then catch KafkaException. Let's not reorder these unless there is a good reason to. It also becomes harder to review since I can't easily see the actual changes vs. reordering.

Also you renamed all exception 'e' to 'ex', is that necessary?

_log.warn("Kafka producer buffer is full, retry in {} ms.", _sendFailureRetryWaitTimeMs, ex);
Thread.sleep(_sendFailureRetryWaitTimeMs);
} catch (KafkaException e) {
Throwable cause = e.getCause();
while (cause instanceof KafkaException) {
cause = cause.getCause();
}
// Set a max_send_attempts for KafkaException as it may be non-recoverable
if (numberOfAttempt > MAX_SEND_ATTEMPTS || ((cause instanceof Error || cause instanceof RuntimeException))) {
_log.error("Send failed for partition {} with a non retriable exception", producerRecord.partition(), e);
throw generateSendFailure(e);
} catch (IllegalStateException ex) {
// The following exception should be quite rare as most exceptions will be throw async callback
_log.warn("Either send is called on a closed producer or broker count is less than minISR, retry in {} ms.",
_sendFailureRetryWaitTimeMs, ex);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like the earlier IllegalStateException block would sleep and you've removed it. why?
Thread.sleep(_sendFailureRetryWaitTimeMs);

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch. This is definitely a bug I introduced. Will revert the entire exception handling piece in send(). Better leave it as it is.

} catch (KafkaException ex) {
Throwable rootCause = ExceptionUtils.getRootCause(ex);
if (numberOfAttempts > MAX_SEND_ATTEMPTS ||
(rootCause instanceof Error || rootCause instanceof RuntimeException)) {
// Set a max_send_attempts for KafkaException as it may be non-recoverable
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment doesn't make sense here, move it back to outside the if condition?

_log.error("Send failed for partition {} with a non retriable exception", producerRecord.partition(), ex);
throw generateSendFailure(ex);
} else {
// The exception might be recoverable. Retry will be attempted
_log.warn("Send failed for partition {} with retriable exception, retry {} out of {} in {} ms.",
producerRecord.partition(), numberOfAttempt, MAX_SEND_ATTEMPTS, _sendFailureRetryWaitTimeMs, e);
producerRecord.partition(), numberOfAttempts, MAX_SEND_ATTEMPTS, _sendFailureRetryWaitTimeMs, ex);
Thread.sleep(_sendFailureRetryWaitTimeMs);
}
} catch (Exception e) {
_log.error("Send failed for partition {} with an exception", producerRecord.partition(), e);
throw generateSendFailure(e);
} catch (Exception ex) {
_log.error("Send failed for partition {} with an exception", producerRecord.partition(), ex);
throw generateSendFailure(ex);
}
}
}

private synchronized void shutdownProducer() {
void doSend(Producer<K, V> producer, ProducerRecord<K, V> record, Callback callback) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this intended to be package private? If so, add a comment, otherwise add the appropriate public/protected/private

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@somandal
I think that's okay; this method's intended to be overriden by another class in the same package. If it's marked protected, it'll be more visible than it needs to be, private wouldn't allow it to be overriden, and public is too permissive.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, makes sense. Just keep things consistent, i.e. if you need a more visible scope for multiple methods, declare them all as package private, and the rest as private. Don't mix protected and package private without having a very good reason to.

producer.send(record, (metadata, exception) -> {
if (exception == null) {
callback.onCompletion(metadata, null);
} else {
callback.onCompletion(metadata, generateSendFailure(exception));
}
});
}

protected synchronized void shutdownProducer() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This can be made package private (no modifier) instead. The same applies to generateSendFailure().

Producer<K, V> producer = _kafkaProducer;
// Nullify first to prevent subsequent send() to use
// the current producer which is being shutdown.
Expand All @@ -246,7 +253,7 @@ private synchronized void shutdownProducer() {
}
}

private DatastreamRuntimeException generateSendFailure(Exception exception) {
protected DatastreamRuntimeException generateSendFailure(Exception exception) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see you overriding this in your Bounded implementation, why make this protected?

_dynamicMetricsManager.createOrUpdateMeter(_metricsNamesPrefix, AGGREGATE, PRODUCER_ERROR, 1);
if (exception instanceof IllegalStateException) {
_log.warn("sent failure transiently, exception: ", exception);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/**
* Copyright 2020 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information.
* See the NOTICE file in the project root for additional information regarding copyright ownership.
*/
package com.linkedin.datastream.common;

import java.time.Duration;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;

import org.apache.commons.lang.NullArgumentException;

import com.google.common.util.concurrent.ThreadFactoryBuilder;

/**
* Utilities for working with CompletableFutures
*/
public class CompletableFutureUtils {
private static final ScheduledExecutorService SCHEDULER = Executors.newScheduledThreadPool(1,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("failAfter-%d").build());

/**
* Returns a CompletableFuture which fails with a TimeoutException after the given interval
* @param duration Duration after which to fail
*/
public static <T> CompletableFuture<T> failAfter(Duration duration) {
final CompletableFuture<T> promise = new CompletableFuture<>();
SCHEDULER.schedule(() -> {
TimeoutException ex = new TimeoutException(String.format("Timeout after {}ms", duration));
return promise.completeExceptionally(ex);
}, duration.toMillis(), TimeUnit.MILLISECONDS);
return promise;
}

/**
* Returns a {@link CompletableFuture} which either successfully executes the given future, or fails with timeout
* after the given duration
* @param future Future to execute
* @param duration Timeout duration
* @throws NullArgumentException
*/
public static <T> CompletableFuture<T> within(CompletableFuture<T> future, Duration duration) throws
NullArgumentException {
if (future == null) {
throw new NullArgumentException("future");
}

CompletableFuture<T> timeout = failAfter(duration);
return future.applyToEither(timeout, Function.identity());
}

}
Loading