Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id DFC71200BCA for ; Mon, 21 Nov 2016 10:58:01 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id DF262160AF9; Mon, 21 Nov 2016 09:58:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 2C590160AEC for ; Mon, 21 Nov 2016 10:58:01 +0100 (CET) Received: (qmail 94848 invoked by uid 500); 21 Nov 2016 09:57:58 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 94626 invoked by uid 99); 21 Nov 2016 09:57:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 21 Nov 2016 09:57:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 8807A2C4C79 for ; Mon, 21 Nov 2016 09:57:58 +0000 (UTC) Date: Mon, 21 Nov 2016 09:57:58 +0000 (UTC) From: "Andrew Efimov (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-4905) Kafka test instability IllegalStateException: Client is not started MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 21 Nov 2016 09:58:02 -0000 [ https://issues.apache.org/jira/browse/FLINK-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15683063#comment-15683063 ] Andrew Efimov commented on FLINK-4905: -------------------------------------- I would suggest the following solution: - not only set null in finally block of {{Kafka08Fetcher}} {{this.zookeeperOffsetHandler = null;}}, also set flag {{volatile closed}} for {{ZookeeperOffsetHandler}}. Threads will check the flag before call methods {{ZookeeperOffsetHandler.setOffsetInZooKeeper}} or {{ZookeeperOffsetHandler.getOffsetFromZooKeeper}} - and create atomic thread counter for ZookeeperOffsetHandler and perform close only if counter = 0, with timeout of cause or use {{CheckpointLock}} that is in context Team, what do you think? > 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)