[ https://issues.apache.org/jira/browse/FLINK-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15680307#comment-15680307 ]
Andrew Efimov commented on FLINK-4905: -------------------------------------- I have started investigation the issue and what I have: 1. InterruptedException in {{Kafka08Fetcher.runFetchLoop}} {noformat} 2 > 08:18:41,705 ERROR org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher - Exception while shutting down consumer threads java.lang.InterruptedException at java.lang.Object.wait(Native Method) at java.lang.Thread.join(Thread.java:1289) at org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher.runFetchLoop(Kafka08Fetcher.java:293) ... {noformat} root cause - it will occur when call t.join on interrupted thread In finally block of the {{Kafka08Fetcher.runFetchLoop}} {{t.join}} is called for threads cancelation procedure, but {{Kafka08Fetcher.runFetchLoop}} is executing in the same thread as {{StreamTask}} which can be interrupted earlier. it is fixed by [~StephanEwen] (10/26/16): {code} // clear the interruption flag // this allows the joining on consumer threads (on best effort) to happen in // case the initial interrupt already Thread.interrupted(); {code} There are several problems in the interaction with {{ZookeeperOffsetHandler}}: 2. I agree with [~tzulitai], work with object {{ZookeeperOffsetHandler}} in {{Kafka08Fetcher}} is not assured thread safe: ok, there are synchronization points in {{StreamTask}}, but what happens in case of failure in {{runFetchLoop}} {{StreamTask}} lock will be free, because {{run()}} is performing, during performing finally block another thread can call {{notifyCheckpointComplete}} and acquire lock although close method is already called on {{ZookeeperOffsetHandler}}. 3. {{ZookeeperOffsetHandler}} is given into {{PeriodicOffsetCommitter}} thread which is closing without thread join, also there is possibility to call {{prepareAndCommitOffsets}} although close method is already called on {{ZookeeperOffsetHandler}}. > 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 > Labels: test-stability > > 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)