[
https://issues.apache.org/jira/browse/KAFKA-6026?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16197126#comment-16197126
]
ASF GitHub Bot commented on KAFKA-6026:
---------------------------------------
GitHub user bartdevylder opened a pull request:
https://github.com/apache/kafka/pull/4044
KAFKA-6026: Fix for indefinite wait in KafkaFutureImpl
Not passing 0 to wait in KafkaFuture implementation. The case where the
timeout argument is 0 is already taken care of in the definition of waitTimeMs
which is at least 1.
The contribution is my original work and I license the work to the project
under the project's open source license
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/bartdevylder/kafka KAFKA-6026
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/kafka/pull/4044.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #4044
----
commit d0962f11e7548e28f75da3d1ee8763489f1f2ad2
Author: bartdevylder <[email protected]>
Date: 2017-10-09T15:05:01Z
Fix for indefinite wait in KafkaFutureImpl
[KAFKA-6026] Not passing 0 to wait in KafkaFuture implementation
----
> Timeout on KafkaFuture not honoured
> -----------------------------------
>
> Key: KAFKA-6026
> URL: https://issues.apache.org/jira/browse/KAFKA-6026
> Project: Kafka
> Issue Type: Bug
> Components: admin
> Affects Versions: 0.11.0.0
> Reporter: Bart De Vylder
>
> I would expect the following code to raise an Exception, either in the
> adminClient creation or a TimeoutException when getting the future (there is
> no kafka running on localhost on that port).
> {code:java}
> Properties config = new Properties();
> config.setProperty("bootstrap.servers", "localhost:1234");
> AdminClient admin = AdminClient.create(config);
> admin.listTopics().names().get(1, TimeUnit.SECONDS);
> {code}
> The code however seems to hang forever in the last step.
> A possible cause for the behavior might be a bug in the KafkaFutureImpl class:
> {code:java}
> private static class SingleWaiter<R> extends BiConsumer<R, Throwable> {
> [...]
> R await(long timeout, TimeUnit unit)
> throws InterruptedException, ExecutionException,
> TimeoutException {
> long startMs = System.currentTimeMillis();
> long waitTimeMs = (unit.toMillis(timeout) > 0) ?
> unit.toMillis(timeout) : 1;
> long delta = 0;
> synchronized (this) {
> while (true) {
> if (exception != null)
> wrapAndThrow(exception);
> if (done)
> return value;
> if (delta > waitTimeMs) {
> throw new TimeoutException();
> }
> this.wait(waitTimeMs - delta);
> delta = System.currentTimeMillis() - startMs;
> }
> }
> }
> {code}
> While debugging I observed {{waitTimeMs}} and {{delta}} to become equal after
> one iteration, giving a {{this.wait(0)}} in the next iteration, which
> according to the documentation
> http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html#wait-long-
> results in an indefinite wait.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)