[ https://issues.apache.org/jira/browse/FLINK-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15765735#comment-15765735 ]
ASF GitHub Bot commented on FLINK-4905: --------------------------------------- GitHub user BrainLogic opened a pull request: https://github.com/apache/flink/pull/3035 [ FLINK-4905] Kafka test instability IllegalStateException: Client is not started Root cause of the issue: `notifyCheckpointComplete` can occur during the cancellation or `runFetchLoop` fail and call `commitOffset` on closed `curatorClient`, so use `CheckpointLock` to close `curatorClient`. There is a diagram in the jira that describes behaviour of using `Kafka08Fetcher`. Notes: 1. I don't like approach where `checkPointLock` is leaked into `SourceContext`, this may lead to deadlock. 2. Work with `ZookeeperOffsetHandler` can continue even after the call Kafka08Fetcher.cancel until the `Handler` will not be null. 3. `ZookeeperOffsetHandler` could have `ReadWriteLock` and use `writeLock` only for close operation, but I have doubt, Flink code base does not contain any `ReentrantLocks`. There is possibility to implement such logic without any locks by using lock-free approach. 4. Also in **jdk8**, we have powerful tool `StampedLock`. In which version of Flink we will be able to use **jdk8** features? You can merge this pull request into a Git repository by running: $ git pull https://github.com/BrainLogic/flink FLINK-4905_PR Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3035.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 #3035 ---- commit fd5524eb15ec772f95c4168818264c63e45f5784 Author: cube <aefimov.c...@gmail.com> Date: 2016-12-20T23:41:12Z [FLINK-4905] Kafka test instability IllegalStateException: Client is not started ---- > Kafka test instability IllegalStateException: Client is not started > ------------------------------------------------------------------- > > Key: FLINK-4905 > URL: https://issues.apache.org/jira/browse/FLINK-4905 > Project: Flink > Issue Type: Bug > Components: Kafka Connector > Reporter: Robert Metzger > Assignee: Andrew Efimov > Labels: test-stability > Attachments: Kafka08Fetcher.png > > > The following travis build > (https://s3.amazonaws.com/archive.travis-ci.org/jobs/170365439/log.txt) > failed because of this error > {code} > 08:17:11,239 INFO org.apache.flink.runtime.jobmanager.JobManager > - Status of job 33ebdc0e7c91be186d80658ce3d17069 (Read some records to > commit offsets to Kafka) changed to FAILING. > java.lang.RuntimeException: Error while confirming checkpoint > at org.apache.flink.runtime.taskmanager.Task$4.run(Task.java:1040) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.IllegalStateException: Client is not started > at > org.apache.flink.shaded.com.google.common.base.Preconditions.checkState(Preconditions.java:173) > at > org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:113) > at > org.apache.curator.utils.EnsurePath$InitialHelper$1.call(EnsurePath.java:148) > at org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:107) > at > org.apache.curator.utils.EnsurePath$InitialHelper.ensure(EnsurePath.java:141) > at org.apache.curator.utils.EnsurePath.ensure(EnsurePath.java:99) > at > org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:133) > at > org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.prepareAndCommitOffsets(ZookeeperOffsetHandler.java:93) > at > org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher.commitInternalOffsetsToKafka(Kafka08Fetcher.java:341) > at > org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.notifyCheckpointComplete(FlinkKafkaConsumerBase.java:421) > at > org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyOfCompletedCheckpoint(AbstractUdfStreamOperator.java:229) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:571) > at org.apache.flink.runtime.taskmanager.Task$4.run(Task.java:1035) > ... 5 more > 08:17:11,241 INFO org.apache.flink.runtime.taskmanager.Task > - Attempting to cancel task Source: Custom Source -> Map -> Map -> Sink: > Unnamed (1/3) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)