Hi Mike,

the job seems to run. It might indeed only be a problem with shutting down
the ZooKeeper utils on the client side after the job has been submitted. I
will try to reproduce it locally. Keep us posted on the state of
CURATOR-466 if something should change.

Cheers,
Till

On Thu, Nov 8, 2018 at 11:17 PM Mikhail Pryakhin <m.prya...@gmail.com>
wrote:

> Hi Till.
> Of course, please find the job bootstrap and YarnJobClusterEntrypoint logs
> attached.
>
> The stacktrace below resembles the bug in Apache Curator
> https://issues.apache.org/jira/browse/CURATOR-466.
>
> java.lang.IllegalStateException: instance must be started before calling
> this method
> at
> org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.base.Preconditions.checkState(Preconditions.java:176)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.getData(CuratorFrameworkImpl.java:375)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.processBackgroundResult(NodeCache.java:288)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.access$300(NodeCache.java:56)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache$3.processResult(NodeCache.java:122)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:749)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:522)
> at
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.ExistsBuilderImpl$1.processResult(ExistsBuilderImpl.java:137)
> at
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:554)
> at
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:505)
>
>
> Kind Regards,
> Mike Pryakhin
>
> On 8 Nov 2018, at 12:12, Till Rohrmann <trohrm...@apache.org> wrote:
>
> Hi Mike,
>
> could you also send me the YarnJobClusterEntrypoint logs. Thanks!
>
> Cheers,
> Till
>
> On Wed, Nov 7, 2018 at 9:27 PM Mikhail Pryakhin <m.prya...@gmail.com>
> wrote:
>
>> Hi Till,
>> Thank you for your reply.
>> Yes, I’ve upgraded to the latest Flink-1.6.2 and the problem is still
>> there, please find the log file attached.
>>
>>
>> Kind Regards,
>> Mike Pryakhin
>>
>> On 7 Nov 2018, at 18:46, Till Rohrmann <trohrm...@apache.org> wrote:
>>
>> Hi Mike,
>>
>> have you tried whether the problem also occurs with Flink 1.6.2? If yes,
>> then please share with us the Flink logs with DEBUG log level to further
>> debug the problem.
>>
>> Cheers,
>> Till
>>
>> On Fri, Oct 26, 2018 at 5:46 PM Mikhail Pryakhin <m.prya...@gmail.com>
>> wrote:
>>
>>> Hi community!
>>>
>>> Righ after I've upgraded flink up to flink-1.6.1 I get an exception
>>> during job deployment as a YARN cluster.
>>> The job is submitted with zookeper HA enabled, in detached mode.
>>>
>>> The flink yaml contains the following properties:
>>>
>>> high-availability: zookeeper
>>> high-availability.zookeeper.quorum: <a list of zookeeper hosts>
>>> high-availability.zookeeper.storageDir: hdfs:///<recovery-folder-path>
>>> high-availability.zookeeper.path.root: <flink-root-path>
>>> high-availability.zookeeper.path.namespace: <flink-job-name>
>>>
>>> the job is deployed via flink CLI command like the following:
>>>
>>> "${FLINK_HOME}/bin/flink" run \
>>> -m yarn-cluster \
>>>     -ynm "${JOB_NAME}-${JOB_VERSION}" \
>>>     -yn "${tm_containers}" \
>>>     -ys "${tm_slots}" \
>>>     -ytm "${tm_memory}" \
>>>     -yjm "${jm_memory}" \
>>>     -p "${parallelism}" \
>>>     -yqu "${queue}" \
>>>     -yt "${YARN_APP_PATH}" \
>>>     -c "${MAIN_CLASS}" \
>>>     -yst \
>>>     -yd \
>>>     ${class_path} \
>>>     "${YARN_APP_PATH}"/"${APP_JAR}"
>>>
>>>
>>> After the job has been successfully deplyed, I've got an exception:
>>>
>>> 2018-10-26 18:29:17,781 | ERROR | Curator-Framework-0 |
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>>> | Background exception was not retry-able or retry gave up
>>> java.lang.InterruptedException
>>> at java.lang.Object.wait(Native Method)
>>> at java.lang.Object.wait(Object.java:502)
>>> at
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1406)
>>> at
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1097)
>>> at
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1130)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths.mkdirs(ZKPaths.java:274)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CreateBuilderImpl$7.performBackgroundOperation(CreateBuilderImpl.java:561)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.OperationAndData.callPerformBackgroundOperation(OperationAndData.java:72)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:831)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.backgroundOperationsLoop(CuratorFrameworkImpl.java:809)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.access$300(CuratorFrameworkImpl.java:64)
>>> at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl$4.call(CuratorFrameworkImpl.java:267)
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>> If the job is deployed in "attached mode" everything goes fine.
>>>
>>>
>>>
>>>
>>>
>>> Kind Regards,
>>> Mike Pryakhin
>>>
>>>
>>
>

Reply via email to