flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Robert Metzger (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8035) Unable to submit job when HA is enabled
Date Wed, 08 Nov 2017 19:37:00 GMT

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

Robert Metzger commented on FLINK-8035:
---------------------------------------

The problem actually also occurred with Flink 1.3.2, but there, the error reporting is better:
{code}
017-11-08 20:29:47,375 WARN  org.apache.zookeeper.ClientCnxn                             
 - Session 0x15f9c132b170016 for server localhost/0:0:0:0:0:0:0:1:2181, unexpected error,
closing socket connection and attempting reconnect
java.io.IOException: Xid out of order. Got Xid 56 with err 0 expected Xid 55 for a packet
with details: clientPath:null serverPath:null finished:false header:: 55,14  replyHeader::
0,0,-4  request:: org.apache.zookeeper.MultiTransactionRecord@7677f7ec response:: org.apache.zookeeper.MultiResponse@0
	at org.apache.zookeeper.ClientCnxn$SendThread.readResponse(ClientCnxn.java:798)
	at org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:94)
	at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:366)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
2017-11-08 20:29:47,480 INFO  org.apache.flink.shaded.org.apache.curator.framework.state.ConnectionStateManager
 - State change: SUSPENDED
{code}

Upgrading the ZK server to 3.4.9 resolved the problem for 1.3.2.
I still think the error handling in 1.4.0 needs to improve (job switching to failed? + an
exception being logged). It would also be good to find out why ZK 3.3.6 didn't work

> Unable to submit job when HA is enabled
> ---------------------------------------
>
>                 Key: FLINK-8035
>                 URL: https://issues.apache.org/jira/browse/FLINK-8035
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager
>    Affects Versions: 1.4.0
>         Environment: Mac OS X
>            Reporter: Robert Metzger
>            Priority: Critical
>
> Steps to reproduce:
> - Get Flink 1.4 (f5a0b4bdfb)
> - Get ZK (3.3.6 in this case)
> - Put the following flink-conf.yaml:
> {code}
> high-availability: zookeeper
> high-availability.storageDir: file:///tmp/flink-ha
> high-availability.zookeeper.quorum: localhost:2181
> high-availability.zookeeper.path.cluster-id: /my-namespace
> {code}
> - Start Flink, submit a job (any streaming example will do)
> The job submission will time out. On the JobManager, it seems that the job submission
gets stuck when trying to submit something to Zookeeper.
> In the JM UI, the job will sit there in status "CREATED"



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message