-
Notifications
You must be signed in to change notification settings - Fork 138
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
jzakaryan
wants to merge
6
commits into
linkedin:master
Choose a base branch
from
jzakaryan:FTR-KafkaClientTimeouts
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+268
−24
Open
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
fe19a50
Added timeouts on send and flush calls in KafkaProducerWrapper
abf10d5
Merge branch 'master' into FTR-KafkaClientTimeouts
3134d6f
Fixed conflicts and error handling logic in flush after merge
988d8a5
Made a separate abstraction for BoundedKafkaProducerWrapper. Added co…
1b8571e
Fixed access modifier issues. Handled flush cancellation better
9a11735
Removed KafkaClientException class. Minor fixes in BoundedKafkaProduc…
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
103 changes: 103 additions & 0 deletions
103
...stream-kafka/src/main/java/com/linkedin/datastream/kafka/BoundedKafkaProducerWrapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/** | ||
* 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.ExecutionException; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.TimeoutException; | ||
|
||
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 configurable timeouts for flush and send calls | ||
*/ | ||
class BoundedKafkaProducerWrapper<K, V> extends KafkaProducerWrapper<K, V> { | ||
private static final long DEFAULT_SEND_TIME_OUT_MS = Duration.ofSeconds(5).toMillis(); | ||
private static final long DEFAULT_FLUSH_TIME_OUT_MS = Duration.ofMinutes(10).toMillis(); | ||
|
||
private static final String SEND_TIMEOUT_CONFIG_KEY = "sendTimeout"; | ||
private static final String FLUSH_TIMEOUT_CONFIG_KEY = "flushTimeout"; | ||
|
||
private final long _sendTimeoutMs; | ||
private final long _flushTimeoutMs; | ||
|
||
BoundedKafkaProducerWrapper(String logSuffix, Properties props, String metricsNamesPrefix) { | ||
super(logSuffix, props, metricsNamesPrefix); | ||
|
||
VerifiableProperties properties = new VerifiableProperties(props); | ||
_sendTimeoutMs = properties.getLong(SEND_TIMEOUT_CONFIG_KEY, DEFAULT_SEND_TIME_OUT_MS); | ||
_flushTimeoutMs = properties.getLong(FLUSH_TIMEOUT_CONFIG_KEY, DEFAULT_FLUSH_TIME_OUT_MS); | ||
} | ||
|
||
@Override | ||
void doSend(Producer<K, V> producer, ProducerRecord<K, V> record, Callback callback) { | ||
CompletableFutureUtils.within(produceMessage(producer, record), Duration.ofMillis(_sendTimeoutMs)) | ||
.thenAccept(m -> callback.onCompletion(m, null)) | ||
.exceptionally(completionEx -> { | ||
Exception cause = (Exception) completionEx.getCause(); | ||
if (cause instanceof java.util.concurrent.TimeoutException) { | ||
_log.warn("KafkaProducerWrapper send timed out. The destination topic may be unavailable."); | ||
} | ||
|
||
callback.onCompletion(null, 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(exception); | ||
} | ||
}); | ||
|
||
return future; | ||
} | ||
|
||
@Override | ||
synchronized void flush() { | ||
ExecutorService executor = Executors.newSingleThreadExecutor(); | ||
Future<?> future = executor.submit(super::flush); | ||
|
||
try { | ||
future.get(_flushTimeoutMs, TimeUnit.MILLISECONDS); | ||
} catch (TimeoutException ex) { | ||
_log.warn("Flush call timed out after {}ms. Cancelling flush", _flushTimeoutMs); | ||
future.cancel(true); | ||
} catch (ExecutionException ex) { | ||
Throwable cause = ex.getCause(); | ||
|
||
if (cause instanceof InterruptException) { | ||
throw (InterruptException) cause; | ||
} else { | ||
// This shouldn't happen | ||
_log.warn("Flush failed.", cause); | ||
} | ||
} catch (InterruptedException ex) { | ||
// This also shouldn't happen because kafka flush use their own InterruptException | ||
Thread.currentThread().interrupt(); | ||
throw new RuntimeException(ex); | ||
} | ||
} | ||
} | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
57 changes: 57 additions & 0 deletions
57
datastream-utils/src/main/java/com/linkedin/datastream/common/CompletableFutureUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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()); | ||
} | ||
|
||
} |
72 changes: 72 additions & 0 deletions
72
...stream-utils/src/test/java/com/linkedin/datastream/common/TestCompletableFutureUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/** | ||
* 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.TimeoutException; | ||
|
||
import org.apache.commons.lang.NullArgumentException; | ||
import org.testng.Assert; | ||
import org.testng.annotations.Test; | ||
|
||
|
||
/** | ||
* Tests for {@link CompletableFutureUtils} | ||
*/ | ||
public class TestCompletableFutureUtils { | ||
|
||
@Test | ||
public void withinTimesOutAfterDuration() { | ||
CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> { | ||
try { | ||
Thread.sleep(10000); | ||
} catch (Exception ex) { | ||
throw new RuntimeException(); | ||
} | ||
return null; | ||
}); | ||
|
||
boolean exception = false; | ||
try { | ||
CompletableFutureUtils.within(future, Duration.ofMillis(10)).join(); | ||
} catch (Exception ex) { | ||
Throwable cause = ex.getCause(); | ||
Assert.assertTrue(cause instanceof TimeoutException); | ||
exception = true; | ||
} | ||
|
||
Assert.assertTrue(exception); | ||
} | ||
|
||
@Test | ||
public void failAfterActuallyFailsAfterDuration() { | ||
CompletableFuture<String> future = CompletableFutureUtils.failAfter(Duration.ofMillis(10)); | ||
|
||
boolean exception = false; | ||
try { | ||
future.join(); | ||
} catch (Exception ex) { | ||
Throwable cause = ex.getCause(); | ||
Assert.assertTrue(cause instanceof TimeoutException); | ||
exception = true; | ||
} | ||
Assert.assertTrue(exception); | ||
} | ||
|
||
@Test | ||
public void withinThrowsNullArgumentExceptionIfNoFutureProvided() { | ||
boolean exception = false; | ||
|
||
try { | ||
CompletableFutureUtils.within(null, Duration.ofMillis(100)); | ||
} catch (NullArgumentException ex) { | ||
exception = true; | ||
} | ||
|
||
Assert.assertTrue(exception); | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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, andpublic
is too permissive.There was a problem hiding this comment.
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.