Ok, I will check.

On Tue, Jun 26, 2018, 12:39 PM Gary Yao <g...@data-artisans.com> wrote:

> Hi Vishal,
>
> You should check the contents of znode /flink_test/[...]/rest_server_lock
> to see
> if the URL is correct.
>
> The host and port should be logged by the RestClient [1]. If you do not
> see the
> message "Sending request of class [...]]" on DEBUG level, probably the
> client is
> not able to get the address from ZK in the first place.
>
> Lastly, the stacktrace you posted seems to be cut off. Can you attach the
> complete client logs?
>
> Best,
> Gary
>
> [1]
> https://github.com/apache/flink/blob/release-1.5.0-rc6/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java#L156
>
> On Tue, Jun 26, 2018 at 4:09 PM, Vishal Santoshi <
> vishal.santo...@gmail.com> wrote:
>
>> The leader znode is the right one  ( it is a binary )
>>
>> get
>> /flink_test/da_15/leader/00000000000000000000000000000000/job_manager_lock
>>
>> wFDakka.tcp://
>> fl...@flink-9edd15d7.bf2.tumblr.net:22161/user/jobmanagersrjava.util.UUIDm/J
>>
>>
>>                         leastSigBitsJ
>>
>>
>>                                     mostSigBitsxpHv
>>
>>
>> So it does ( I think ) resolve the right leader of the HA, but from there
>> ( the logs do not help as DEBUG logs do not expose what server it hits
>> sadly ) .
>>
>>
>> On Tue, Jun 26, 2018 at 9:57 AM, Vishal Santoshi <
>> vishal.santo...@gmail.com> wrote:
>>
>>> OK few things
>>>
>>> 2018-06-26 13:31:29 INFO  CliFrontend:282 -  Starting Command Line
>>> Client (Version: 1.5.0, Rev:c61b108, Date:24.05.2018 @ 14:54:44 UTC)
>>>
>>> ...
>>>
>>> 2018-06-26 13:31:31 INFO  ClientCnxn:876 - Socket connection
>>> established to zk-f1fb95b9.bf2.tumblr.net/10.246.218.17:2181,
>>> initiating session
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:949 - Session establishment request
>>> sent on zk-f1fb95b9.bf2.tumblr.net/10.246.218.17:2181
>>>
>>> 2018-06-26 13:31:31 INFO  ClientCnxn:1299 - Session establishment
>>> complete on server zk-f1fb95b9.bf2.tumblr.net/10.246.218.17:2181,
>>> sessionid = 0x35add547801ea07, negotiated timeout = 40000
>>>
>>> 2018-06-26 13:31:31 INFO  RestClient:119 - Rest client endpoint started.
>>>
>>> 2018-06-26 13:31:31 INFO  ZooKeeperLeaderRetrievalService:100 -
>>> Starting ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 1,3  replyHeader:: 1,60416530560,0  request::
>>> '/flink_test,F  response::
>>> s{47265479496,47265479496,1489163688703,1489163688703,0,2,0,0,0,2,60416492885}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 2,3  replyHeader:: 2,60416530560,0  request::
>>> '/flink_test/da_15,F  response::
>>> s{60416492885,60416492885,1529755199131,1529755199131,0,5,0,0,0,5,60416521584}
>>>
>>>
>>> 2018-06-26 13:31:31 INFO  ZooKeeperLeaderRetrievalService:100 -
>>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 3,3  replyHeader:: 3,60416530560,0  request::
>>> '/flink_test,F  response::
>>> s{47265479496,47265479496,1489163688703,1489163688703,0,2,0,0,0,2,60416492885}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 4,3  replyHeader:: 4,60416530560,0  request::
>>> '/flink_test/da_15,F  response::
>>> s{60416492885,60416492885,1529755199131,1529755199131,0,5,0,0,0,5,60416521584}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 5,3  replyHeader:: 5,60416530560,0  request::
>>> '/flink_test/da_15/leader,F  response::
>>> s{60416492887,60416492887,1529755199191,1529755199191,0,1,0,0,0,1,60416492888}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet::
>>> clientPath:/flink_test/da_15/leader/rest_server_lock
>>> serverPath:/flink_test/da_15/leader/rest_server_lock finished:false
>>> header:: 6,3  replyHeader:: 6,60416530560,-101  request::
>>> '/flink_test/da_15/leader/rest_server_lock,T  response::
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 7,3  replyHeader:: 7,60416530560,0  request::
>>> '/flink_test,F  response::
>>> s{47265479496,47265479496,1489163688703,1489163688703,0,2,0,0,0,2,60416492885}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 8,3  replyHeader:: 8,60416530560,0  request::
>>> '/flink_test/da_15,F  response::
>>> s{60416492885,60416492885,1529755199131,1529755199131,0,5,0,0,0,5,60416521584}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 9,3  replyHeader:: 9,60416530560,0  request::
>>> '/flink_test/da_15/leader,F  response::
>>> s{60416492887,60416492887,1529755199191,1529755199191,0,1,0,0,0,1,60416492888}
>>>
>>>
>>> 2018-06-26 13:31:31 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet::
>>> clientPath:/flink_test/da_15/leader/dispatcher_lock
>>> serverPath:/flink_test/da_15/leader/dispatcher_lock finished:false header::
>>> 10,3  replyHeader:: 10,60416530560,-101  request::
>>> '/flink_test/da_15/leader/dispatcher_lock,T  response::
>>>
>>> 2018-06-26 13:31:31 INFO  CliFrontend:914 - Waiting for response...
>>>
>>> Waiting for response...
>>>
>>> 2018-06-26 13:31:44 DEBUG ClientCnxn:742 - Got ping response for
>>> sessionid: 0x35add547801ea07 after 0ms
>>>
>>> 2018-06-26 13:31:58 DEBUG ClientCnxn:742 - Got ping response for
>>> sessionid: 0x35add547801ea07 after 0ms
>>>
>>> 2018-06-26 13:32:01 INFO  RestClient:123 - Shutting down rest endpoint.
>>>
>>> 2018-06-26 13:32:01 INFO  RestClient:140 - Rest endpoint shutdown
>>> complete.
>>>
>>> 2018-06-26 13:32:01 INFO  ZooKeeperLeaderRetrievalService:117 -
>>> Stopping ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>>
>>> 2018-06-26 13:32:01 INFO  ZooKeeperLeaderRetrievalService:117 -
>>> Stopping ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>
>>> 2018-06-26 13:32:01 DEBUG CuratorFrameworkImpl:282 - Closing
>>>
>>> 2018-06-26 13:32:01 INFO  CuratorFrameworkImpl:821 -
>>> backgroundOperationsLoop exiting
>>>
>>> 2018-06-26 13:32:01 DEBUG CuratorZookeeperClient:199 - Closing
>>>
>>> 2018-06-26 13:32:01 DEBUG ConnectionState:115 - Closing
>>>
>>> 2018-06-26 13:32:01 DEBUG ZooKeeper:673 - Closing session:
>>> 0x35add547801ea07
>>>
>>> 2018-06-26 13:32:01 DEBUG ClientCnxn:1370 - Closing client for session:
>>> 0x35add547801ea07
>>>
>>> 2018-06-26 13:32:01 DEBUG ClientCnxn:843 - Reading reply
>>> sessionid:0x35add547801ea07, packet:: clientPath:null serverPath:null
>>> finished:false header:: 11,-11  replyHeader:: 11,60416530561,0  request::
>>> null response:: null
>>>
>>> 2018-06-26 13:32:01 DEBUG ClientCnxn:1354 - Disconnecting client for
>>> session: 0x35add547801ea07
>>>
>>> 2018-06-26 13:32:01 INFO  ZooKeeper:684 - Session: 0x35add547801ea07
>>> closed
>>>
>>> 2018-06-26 13:32:01 INFO  ClientCnxn:519 - EventThread shut down for
>>> session: 0x35add547801ea07
>>>
>>> 2018-06-26 13:32:01 DEBUG ClientCnxn:1146 - An exception was thrown
>>> while closing send thread for session 0x35add547801ea07 : Unable to read
>>> additional data from server sessionid 0x35add547801ea07, likely server has
>>> closed socket
>>>
>>> 2018-06-26 13:32:01 ERROR CliFrontend:891 - Error while running the
>>> command.
>>>
>>> org.apache.flink.util.FlinkException: Failed to retrieve job list.
>>>
>>> at org.apache.flink.client.cli.Cl
>>> iFrontend.listJobs(CliFrontend.java:429)
>>>
>>> at org.apache.flink.client.cli.Cl
>>> iFrontend.lambda$list$0(CliFrontend.java:412)
>>>
>>>
>>> On Tue, Jun 26, 2018 at 5:43 AM, zhangminglei <18717838...@163.com>
>>> wrote:
>>>
>>>> By the way, in HA set up.
>>>>
>>>> 在 2018年6月26日,下午5:39,zhangminglei <18717838...@163.com> 写道:
>>>>
>>>> Hi, Gary Yao
>>>>
>>>> Once I discovered that there was a change in the ip address[
>>>> jobmanager.rpc.address ]. From 10.208.73.129 to localhost. I think that
>>>> will cause the issue. What do you think ?
>>>>
>>>> Cheers
>>>> Minglei
>>>>
>>>> 在 2018年6月26日,下午4:53,Gary Yao <g...@data-artisans.com> 写道:
>>>>
>>>> Hi Vishal,
>>>>
>>>> Could it be that you are not using the 1.5.0 client? The stacktrace you
>>>> posted
>>>> does not reference valid lines of code in the release-1.5.0-rc6 tag.
>>>>
>>>> If you have a HA setup, the host and port of the leading JM will be
>>>> looked up
>>>> from ZooKeeper before job submission. Therefore, the flink-conf.yaml
>>>> used by the
>>>> client must have the same ZooKeeper configuration as used by the Flink
>>>> cluster.
>>>>
>>>> Best,
>>>> Gary
>>>>
>>>> On Mon, Jun 25, 2018 at 5:32 PM, Vishal Santoshi <
>>>> vishal.santo...@gmail.com> wrote:
>>>>
>>>>> I think all I need to add is
>>>>>
>>>>> web.port: 8081
>>>>> rest.port: 8081
>>>>>
>>>>> to the JM flink conf ?
>>>>>
>>>>> On Mon, Jun 25, 2018 at 10:46 AM, Vishal Santoshi <
>>>>> vishal.santo...@gmail.com> wrote:
>>>>>
>>>>>> Another issue I saw with flink cli...
>>>>>>
>>>>>> org.apache.flink.client.program.ProgramInvocationException: The
>>>>>> program execution failed: JobManager did not respond within 120000 ms
>>>>>> at
>>>>>> org.apache.flink.client.program.ClusterClient.runDetached(ClusterClient.java:524)
>>>>>> at
>>>>>> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:103)
>>>>>> at
>>>>>> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:456)
>>>>>> at
>>>>>> org.apache.flink.client.program.DetachedEnvironment.finalizeExecute(DetachedEnvironment.java:77)
>>>>>> at org.apach
>>>>>>
>>>>>> This was a simple submission  and it does succeed through the UI.
>>>>>>
>>>>>> Has there been a regression on CLI... I could not find any
>>>>>> documentation around it.
>>>>>>
>>>>>> I have a HA JM setup.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Jun 25, 2018 at 10:22 AM, Chesnay Schepler <
>>>>>> ches...@apache.org> wrote:
>>>>>>
>>>>>>> The watermark issue is know and will be fixed in 1.5.1
>>>>>>>
>>>>>>>
>>>>>>> On 25.06.2018 15:03, Vishal Santoshi wrote:
>>>>>>>
>>>>>>> Thank you....
>>>>>>>
>>>>>>> One addition
>>>>>>>
>>>>>>> I do not see WM info on the UI  ( Attached )
>>>>>>>
>>>>>>> Is this a know issue. The same pipe on our production has the WM (
>>>>>>> In fact never had an issue with  Watermarks not appearing ) . Am I 
>>>>>>> missing
>>>>>>> something ?
>>>>>>>
>>>>>>> On Mon, Jun 25, 2018 at 4:15 AM, Fabian Hueske <fhue...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Vishal,
>>>>>>>>
>>>>>>>> 1. I don't think a rolling update is possible. Flink 1.5.0 changed
>>>>>>>> the process orchestration and how they communicate. IMO, the way to go 
>>>>>>>> is
>>>>>>>> to start a Flink 1.5.0 cluster, take a savepoint on the running job, 
>>>>>>>> start
>>>>>>>> from the savepoint on the new cluster and shut the old job down.
>>>>>>>> 2. Savepoints should be compatible.
>>>>>>>> 3. You can keep the slot configuration as before.
>>>>>>>> 4. As I said before, mixing 1.5 and 1.4 processes does not work (or
>>>>>>>> at least, it was not considered a design goal and nobody paid attention
>>>>>>>> that it is possible).
>>>>>>>>
>>>>>>>> Best, Fabian
>>>>>>>>
>>>>>>>>
>>>>>>>> 2018-06-23 13:38 GMT+02:00 Vishal Santoshi <
>>>>>>>> vishal.santo...@gmail.com>:
>>>>>>>>
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> Can or has any one  done  a rolling upgrade from 1.4 to 1.5 ?  I
>>>>>>>>> am not sure we can. It seems that JM cannot recover jobs with this
>>>>>>>>> exception
>>>>>>>>>
>>>>>>>>> Caused by: java.io.InvalidClassException:
>>>>>>>>> org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
>>>>>>>>> local class incompatible: stream classdesc serialVersionUID =
>>>>>>>>> -647384516034982626, local class serialVersionUID = 2
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> Does SP on 1.4, resume on 1.5 ( pretty basic but no harm asking ) ?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>>
>>>>>>>>> https://ci.apache.org/projects/flink/flink-docs-release-1.5/release-notes/flink-1.5.html#update-configuration-for-reworked-job-deployment
>>>>>>>>> The taskmanager.numberOfTaskSlots: What would be the desired
>>>>>>>>> setting in a stand alone ( non mesos/yarn ) cluster ?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 4. I suspend all jobs and establish 1.5 on the JM ( the TMs are
>>>>>>>>> still running with 1.4 ) . JM refuse to start  with
>>>>>>>>>
>>>>>>>>> Jun 23 07:34:23 flink-ad21ac07.bf2.tumblr.net docker[3395]:
>>>>>>>>> 2018-06-23 11:34:23 ERROR JobManager:116 - Failed to recover job
>>>>>>>>> 454cd84a519f3b50e88bcb378d8a1330.
>>>>>>>>> Jun 23 07:34:23 flink-ad21ac07.bf2.tumblr.net docker[3395]:
>>>>>>>>> java.lang.InstantiationError: org.apache.flink.runtime.blob.BlobKey
>>>>>>>>> Jun 23 07:34:23 flink-ad21ac07.bf2.tumblr.net docker[3395]: at
>>>>>>>>> sun.reflect.GeneratedSerializationConstructorAccessor51.newInstance(Unknown
>>>>>>>>> Source)
>>>>>>>>> Jun 23 07:34:23 flink-ad21ac07.bf2.tumblr.net docker[3395]: at
>>>>>>>>> java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>>>>>>>>> Jun 23 07:34:23 flink-ad21ac07.bf2.tumblr.net docker[3395]: at
>>>>>>>>> java.io.ObjectStreamClass.newInstance(ObjectStreamClass.java:1079)
>>>>>>>>> Jun
>>>>>>>>> .....
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Any feedback would be highly appreciated...
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>>
>>>
>>
>

Reply via email to