[ 
https://issues.apache.org/jira/browse/FLINK-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15607498#comment-15607498
 ] 

Tzu-Li (Gordon) Tai edited comment on FLINK-4905 at 10/26/16 5:39 AM:
----------------------------------------------------------------------

Had a quick look at this. The only place we're closing the curator client 
ourselves is in the {{finally}} clause of the {{runFetchLoop}} in the 0.8 
Fetcher.
I'm suspecting the cause is that {{notifyCheckpointComplete}} was called before 
the fetch loop entered the finally clause and started the offset committing to 
ZK ({{prepareAndCommitOffsets() call}}), but the curator client was closed 
mid-way once the fetch loop entered finally clause. A good fix perhaps would be 
to add a lock to synchronize {{close()}} and {{prepareAndCommitOffsets()}} / 
{{getCommittedOffsets}} in the ZK offset handler to make sure we aren't called 
close mid way.

>From the test logs the {{testStartFromKafkaCommitOffsets}} seems unrelated to 
>this instability.


was (Author: tzulitai):
Had a quick look at this. The only place we're closing the curator client 
ourselves is in the {{finally}} clause of the {{runFetchLoop}} in the 0.8 
Fetcher.
I'm suspecting the cause is that {{notifyCheckpointComplete}} was called before 
the fetch loop entered the finally clause and started the offset committing to 
ZK, but the curator client was closed mid-way once the fetch loop entered 
finally clause. A good fix perhaps would be to add a lock to synchronize 
{{close()}} and {{prepareAndCommitOffsets()}} / {{getCommittedOffsets}} in the 
ZK offset handler to make sure we aren't called close mid way.

>From the test logs the {{testStartFromKafkaCommitOffsets}} seems unrelated to 
>this instability.

> 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)

Reply via email to