Return-Path: X-Original-To: apmail-flink-user-archive@minotaur.apache.org Delivered-To: apmail-flink-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 78BAD18CF9 for ; Thu, 3 Mar 2016 13:32:22 +0000 (UTC) Received: (qmail 61338 invoked by uid 500); 3 Mar 2016 13:32:22 -0000 Delivered-To: apmail-flink-user-archive@flink.apache.org Received: (qmail 61253 invoked by uid 500); 3 Mar 2016 13:32:22 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@flink.apache.org Delivered-To: mailing list user@flink.apache.org Received: (qmail 61244 invoked by uid 99); 3 Mar 2016 13:32:22 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Mar 2016 13:32:22 +0000 Received: from mail-wm0-f48.google.com (mail-wm0-f48.google.com [74.125.82.48]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id 717D51A00C5 for ; Thu, 3 Mar 2016 13:32:21 +0000 (UTC) Received: by mail-wm0-f48.google.com with SMTP id p65so34961771wmp.1 for ; Thu, 03 Mar 2016 05:32:21 -0800 (PST) X-Gm-Message-State: AD7BkJLL81x7vdO3IdOfJvK5PWX0p1mR1LanVpV0ThooI7hASggOVQI++zwb/pflMX7XMpGLtz0ahXzMwGK/vQ== MIME-Version: 1.0 X-Received: by 10.28.103.3 with SMTP id b3mr5579235wmc.65.1457011940226; Thu, 03 Mar 2016 05:32:20 -0800 (PST) Received: by 10.28.113.129 with HTTP; Thu, 3 Mar 2016 05:32:20 -0800 (PST) In-Reply-To: <4385FC73-9684-46EA-B36F-18691350C009@tngtech.com> References: <4385FC73-9684-46EA-B36F-18691350C009@tngtech.com> Date: Thu, 3 Mar 2016 14:32:20 +0100 X-Gmail-Original-Message-ID: Message-ID: Subject: Re: YARN JobManager HA using wrong network interface From: Till Rohrmann To: user@flink.apache.org Content-Type: multipart/alternative; boundary=001a114a91b2218462052d2507db --001a114a91b2218462052d2507db Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable I've created an issue [1] and opened a PR [2] to fix the issue. [1] https://issues.apache.org/jira/browse/FLINK-3570 [2] https://github.com/apache/flink/pull/1758 Cheers, Till On Thu, Mar 3, 2016 at 12:33 PM, Maximilian Bode < maximilian.bode@tngtech.com> wrote: > Hi Ufuk, Till and Stephan, > > Yes, that is what we observed. The primary hostname, i.e. the one returne= d > by the unix hostname command, is in fact bound to the eth0 interface, > whereas Flink uses the eth1 interface (pertaining to another hostname). > > Changing akka.lookup.timeout to 100 s seems to fix the problem as now the > new job manager is available in sufficient time. I still would agree with > Stephan on taking the local hostname being the preferred strategy. > > Cheers, > Max > =E2=80=94 > Maximilian Bode * Junior Consultant * maximilian.bode@tngtech.com > TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterf=C3=B6hring > Gesch=C3=A4ftsf=C3=BChrer: Henrik Klagges, Christoph Stock, Dr. Robert Da= hlke > Sitz: Unterf=C3=B6hring * Amtsgericht M=C3=BCnchen * HRB 135082 > > Am 03.03.2016 um 12:29 schrieb Till Rohrmann : > > No I don't think this behaviour has been introduced by HA. That is the > default behaviour we used for a long time. If you think we should still > change it, then I can open an issue for it. > > On Thu, Mar 3, 2016 at 12:20 PM, Stephan Ewen wrote: > >> Okay, that is a change from the original behavior, introduced in HA. >> Originally, if the connection attempts failed, it always returned the In= etAddress.getLocalHost() >> interface. >> I think we should change it back to that, because that interface is by >> far the best possible heuristic. >> >> On Thu, Mar 3, 2016 at 11:39 AM, Till Rohrmann >> wrote: >> >>> If I=E2=80=99m not mistaken, then it=E2=80=99s not necessarily true tha= t the heuristic >>> returns InetAddress.getLocalHost() in all cases. The heuristic will >>> select the first network interface with the afore-mentioned conditions = but >>> before returning it, it will try a last time to connect to the JM via t= he >>> interface bound to InetAddress.getLocalHost(). However, if this fails, >>> then the heuristically selected network interface will be returned. >>> =E2=80=8B >>> >>> On Thu, Mar 3, 2016 at 10:49 AM, Stephan Ewen wrote: >>> >>>> If the ThasManager cannot connect to the JobManager, it will use the >>>> interface that is bound to the machine's host name >>>> ("InetAddress.getLocalHost()"). >>>> >>>> So, the best way to fix this would be to make sure that all machines >>>> have a proper network configuration. Then Flink would either use an ad= dress >>>> that can connect (via trying various interfaces), or it would default = back >>>> to the hostname/interface that is configured on the machine. >>>> >>>> >>>> On Thu, Mar 3, 2016 at 10:43 AM, Till Rohrmann >>>> wrote: >>>> >>>>> Hi Max, >>>>> >>>>> the problem is that before starting the TM, we have to find the >>>>> network interface which is reachable by the other machines. So what w= e do >>>>> is to connect to the current JobManager. If it should happen, as in y= our >>>>> case, that the JobManager just died and the new JM address has not be= en >>>>> written to ZooKeeper, then the TMs don=E2=80=99t have much choice oth= er than using >>>>> the heuristic. >>>>> >>>>> I can=E2=80=99t really tell why eth1 is chosen over eth0. The conditi= on is >>>>> that the interface address is an Inet4Address, no link local address >>>>> as well as not a loopback address. >>>>> >>>>> Thus, Ufuk=E2=80=99s solution, to increase akka.lookup.timeout seems = to be >>>>> the easiest solution to solve your problem. I=E2=80=99ve checked the = default value >>>>> is set to 10 s which might be a bit too low for restarting a new JM >>>>> and publishing its address via ZooKeeper. >>>>> >>>>> Cheers, >>>>> Till >>>>> =E2=80=8B >>>>> >>>>> On Thu, Mar 3, 2016 at 10:28 AM, Ufuk Celebi wrote: >>>>> >>>>>> I had an offline chat with Till about this. He pointed out that the >>>>>> address is chosen once at start up time (while not being able to >>>>>> connect to the old job manager) and then it stays fixed at eth1. >>>>>> >>>>>> You can increase the lookup timeout by setting akka.lookup.timeout t= o >>>>>> a higher value (like 100 s). This is the only workaroud I'm aware of >>>>>> at this point. Maybe Till can chime in here whether this has other >>>>>> implications as well? >>>>>> >>>>>> =E2=80=93 Ufuk >>>>>> >>>>>> On Thu, Mar 3, 2016 at 9:59 AM, Ufuk Celebi wrote: >>>>>> > Hey Max! >>>>>> > >>>>>> > for the first WARN in >>>>>> > org.apache.flink.runtime.webmonitor.JobManagerRetriever: this is >>>>>> > expected if the new leader has not updated ZooKeeper yet. The >>>>>> > important thing is that the new leading job manager is eventually >>>>>> > retrieved. This did happen, right? >>>>>> > >>>>>> > Regarding eth1 vs. eth0: After the new job manager becomes leader, >>>>>> the >>>>>> > task manager should re-try connecting to it with the same strategy >>>>>> as >>>>>> > in the initial connection establishment (e.g. try SLOW first and >>>>>> only >>>>>> > fall back to HEURISTIC). Can you see in the logs whether this >>>>>> happens? >>>>>> > >>>>>> > The best thing would be to share the complete logs. Is this >>>>>> possible? >>>>>> > If not publicly, feel free to send it to me privately (uce at apac= he >>>>>> > org). >>>>>> > >>>>>> > =E2=80=93 Ufuk >>>>>> > >>>>>> > >>>>>> > On Thu, Mar 3, 2016 at 9:21 AM, Maximilian Bode >>>>>> > wrote: >>>>>> >> Hi everyone, >>>>>> >> >>>>>> >> we are trying to get to work JobManager HA in the context of a >>>>>> per-job YARN >>>>>> >> session using the 1.0.0-rc3 from a few days ago and are having a >>>>>> problem >>>>>> >> concerning task managers with several network interfaces. >>>>>> >> >>>>>> >> After manually killing the job manager process, the jobmanager.lo= g >>>>>> on the >>>>>> >> newly allocated second job manager reads: >>>>>> >> --- >>>>>> >> 2016-03-02 18:01:09,635 WARN Remoting >>>>>> >> - Tried to associate with unreachable remote address >>>>>> >> [akka.tcp://flink@10.127.68.136:34811]. Address is now gated for >>>>>> 5000 ms, >>>>>> >> all messages to this address will be delivered to dead letters. >>>>>> Reason: >>>>>> >> Connection refused: /10.127.68.136:34811 >>>>>> >> 2016-03-02 18:01:09,644 WARN >>>>>> >> org.apache.flink.runtime.webmonitor.JobManagerRetriever - >>>>>> Failed to >>>>>> >> retrieve leader gateway and port. >>>>>> >> akka.actor.ActorNotFound: Actor not found for: >>>>>> >> ActorSelection[Anchor(akka.tcp://flink@10.127.68.136:34811/), >>>>>> >> Path(/user/jobmanager)] >>>>>> >> at >>>>>> >> >>>>>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection= .scala:65) >>>>>> >> at >>>>>> >> >>>>>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection= .scala:63) >>>>>> >> at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(B= atchingExecutor.scala:67) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(Bat= chingExecutor.scala:82) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingEx= ecutor.scala:59) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingEx= ecutor.scala:59) >>>>>> >> at >>>>>> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:7= 2) >>>>>> >> at >>>>>> akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatche= dExecute(Future.scala:74) >>>>>> >> at >>>>>> akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:= 110) >>>>>> >> at >>>>>> >> >>>>>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(= Future.scala:73) >>>>>> >> at >>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scal= a:40) >>>>>> >> at >>>>>> >> >>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.sca= la:248) >>>>>> >> at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:267) >>>>>> >> at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:508= ) >>>>>> >> at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:541= ) >>>>>> >> at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:531) >>>>>> >> at >>>>>> >> >>>>>> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(Re= moteActorRefProvider.scala:87) >>>>>> >> at akka.remote.EndpointWriter.postStop(Endpoint.scala:561) >>>>>> >> at akka.actor.Actor$class.aroundPostStop(Actor.scala:475) >>>>>> >> at akka.remote.EndpointActor.aroundPostStop(Endpoint.scala:415) >>>>>> >> at >>>>>> >> >>>>>> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandl= ing$$finishTerminate(FaultHandling.scala:210) >>>>>> >> at >>>>>> akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala= :172) >>>>>> >> at akka.actor.ActorCell.terminate(ActorCell.scala:369) >>>>>> >> at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:462) >>>>>> >> at akka.actor.ActorCell.systemInvoke(ActorCell.scala:478) >>>>>> >> at >>>>>> akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:279) >>>>>> >> at akka.dispatch.Mailbox.run(Mailbox.scala:220) >>>>>> >> at akka.dispatch.Mailbox.exec(Mailbox.scala:231) >>>>>> >> at >>>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) >>>>>> >> at >>>>>> >> >>>>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPoo= l.java:1339) >>>>>> >> at >>>>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1= 979) >>>>>> >> at >>>>>> >> >>>>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThr= ead.java:107) >>>>>> >> --- >>>>>> >> where the IP not found is from the old job manager. So far, is >>>>>> this the >>>>>> >> expected behavior? >>>>>> >> >>>>>> >> The problem then arises on a new task manager, which also tries t= o >>>>>> connect >>>>>> >> to the old job manager unsuccessfully. The >>>>>> ZooKeeperLeaderRetrievalService >>>>>> >> starts cycling through the available network interfaces, as can b= e >>>>>> seen in >>>>>> >> the relevant taskmanager.log: >>>>>> >> --- >>>>>> >> 2016-03-02 18:01:13,636 INFO >>>>>> >> >>>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalSer= vice - >>>>>> >> Starting ZooKeeperLeaderRetrievalService. >>>>>> >> 2016-03-02 18:01:13,646 INFO >>>>>> >> org.apache.flink.runtime.util.LeaderRetrievalUtils - >>>>>> Trying to >>>>>> >> select the network interface and address to use by connecting to >>>>>> the leading >>>>>> >> JobManager. >>>>>> >> 2016-03-02 18:01:13,646 INFO >>>>>> >> org.apache.flink.runtime.util.LeaderRetrievalUtils - >>>>>> TaskManager >>>>>> >> will try to connect for 10000 milliseconds before falling back to >>>>>> heuristics >>>>>> >> 2016-03-02 18:01:13,712 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Retrieved new target address /10.127.68.136:34811. >>>>>> >> 2016-03-02 18:01:14,079 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Trying to connect to address /10.127.68.136:34811 >>>>>> >> 2016-03-02 18:01:14,082 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address >>>>>> >> 'task.manager.eth0.hostname.com/10.127.68.136': Connection refuse= d >>>>>> >> 2016-03-02 18:01:14,082 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/10.127.68.136': Connection >>>>>> refused >>>>>> >> 2016-03-02 18:01:14,082 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/10.120.193.110': Connection >>>>>> refused >>>>>> >> 2016-03-02 18:01:14,082 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/10.127.68.136': Connection >>>>>> refused >>>>>> >> 2016-03-02 18:01:14,083 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/127.0.0.1': Connection refused >>>>>> >> 2016-03-02 18:01:14,083 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/10.120.193.110': Connection >>>>>> refused >>>>>> >> 2016-03-02 18:01:14,083 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/10.127.68.136': Connection >>>>>> refused >>>>>> >> 2016-03-02 18:01:14,083 INFO >>>>>> org.apache.flink.runtime.net.ConnectionUtils >>>>>> >> - Failed to connect from address '/127.0.0.1': Connection refused >>>>>> >> --- >>>>>> >> After five repetitions, the task manager stops trying to retrieve >>>>>> the leader >>>>>> >> and using the HEURISTIC strategy ends up using eth1 >>>>>> (10.120.193.110) from >>>>>> >> now on: >>>>>> >> --- >>>>>> >> 2016-03-02 18:01:23,650 INFO >>>>>> >> >>>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalSer= vice - >>>>>> >> Stopping ZooKeeperLeaderRetrievalService. >>>>>> >> 2016-03-02 18:01:23,655 INFO org.apache.zookeeper.ClientCnxn >>>>>> >> - EventThread shut down >>>>>> >> 2016-03-02 18:01:23,655 INFO org.apache.zookeeper.ZooKeeper >>>>>> >> - Session: 0x25229757cff035b closed >>>>>> >> 2016-03-02 18:01:23,664 INFO >>>>>> >> org.apache.flink.runtime.taskmanager.TaskManager - >>>>>> TaskManager >>>>>> >> will use hostname/address 'task.manager.eth1.hostname.com' >>>>>> (10.120.193.110) >>>>>> >> for communication. >>>>>> >> --- >>>>>> >> Following the new jobmanager is discovered and the taskmanager is >>>>>> able to >>>>>> >> register at the jobmanager using eth1. The problem is that >>>>>> connections TO >>>>>> >> eth1 are not possible. So flink should always use eth0. The >>>>>> exception we >>>>>> >> later see is: >>>>>> >> --- >>>>>> >> java.io.IOException: Connecting the channel failed: Connecting to >>>>>> remote >>>>>> >> task manager + 'other.task.manager.eth1.hostname/ >>>>>> 10.120.193.111:46620' has >>>>>> >> failed. This might indicate that the remote task manager has been >>>>>> lost. >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFact= ory$ConnectingChannel.waitForChannel(PartitionRequestClientFactory.java:196= ) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFact= ory$ConnectingChannel.access$000(PartitionRequestClientFactory.java:131) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFact= ory.createPartitionRequestClient(PartitionRequestClientFactory.java:83) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.netty.NettyConnectionManager.cre= atePartitionRequestClient(NettyConnectionManager.java:60) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.partition.consumer.RemoteInputCh= annel.requestSubpartition(RemoteInputChannel.java:115) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.partition.consumer.SingleInputGa= te.requestPartitions(SingleInputGate.java:388) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.runtime.io.network.partition.consumer.SingleInputGa= te.getNextBufferOrEvent(SingleInputGate.java:411) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.streaming.runtime.io.BarrierBuffer.getNextNonBlocke= d(BarrierBuffer.java:108) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processIn= put(StreamInputProcessor.java:175) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneI= nputStreamTask.java:65) >>>>>> >> at >>>>>> >> >>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTas= k.java:224) >>>>>> >> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559) >>>>>> >> at java.lang.Thread.run(Thread.java:744) >>>>>> >> --- >>>>>> >> The root cause seems to be that network interface selection is >>>>>> still using >>>>>> >> the old jobmanager location and hence is not able to choose the >>>>>> right >>>>>> >> interface. In particular, it seems that iteration order over the >>>>>> network >>>>>> >> interfaces differs between the HEURISTIC and SLOW strategy, which >>>>>> then leads >>>>>> >> to the wrong interface being selected. >>>>>> >> >>>>>> >> Cheers, >>>>>> >> Max >>>>>> >> =E2=80=94 >>>>>> >> Maximilian Bode * Junior Consultant * maximilian.bode@tngtech.com >>>>>> >> TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterf=C3=B6h= ring >>>>>> >> Gesch=C3=A4ftsf=C3=BChrer: Henrik Klagges, Christoph Stock, Dr. R= obert Dahlke >>>>>> >> Sitz: Unterf=C3=B6hring * Amtsgericht M=C3=BCnchen * HRB 135082 >>>>>> >> >>>>>> >>>>> >>>>> >>>> >>> >> > > --001a114a91b2218462052d2507db Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
I've created an issue [1] and opened a PR [2] to fix t= he issue.

[1] https://issues.apache.org/jira/browse/FLINK-3570

Cheers,
Till


=
On Thu, Mar 3, 2016 at 12:33 PM, Maximilian Bode= <maximilian.bode@tngtech.com> wrote:
Hi Ufuk, Til= l and Stephan,

Yes, that is what we observed. The prima= ry hostname, i.e. the one returned by the unix hostname command, is in fact= bound to the eth0 interface, whereas Flink uses the eth1 interface (pertai= ning to another hostname).

Changing akka.lookup.timeout = to 100 s seems to fix the problem as now the new job manager is available i= n sufficient time. I still would agree with Stephan on taking the local hos= tname being the preferred strategy.

Cheers,
=C2=A0Max
=E2=80=94=C2=A0
Maximilian Bode *= Junior Consultant * maximilian.bode@tngtech.com
TNG Technology Consulting G= mbH, Betastr. 13a, 85774 Unterf=C3=B6hring
Gesch=C3=A4ftsf=C3=BCh= rer: Henrik Klagges, Christoph Stock, Dr. Robert Dahlke
Sitz: Unt= erf=C3=B6hring * Amtsgericht M=C3=BCnchen * HRB 135082

Am 03= .03.2016 um 12:29 schrieb Till Rohrmann <till.rohrmann@gmail.com>:

No I don't think this behaviour has been introduced = by HA. That is the default behaviour we used for a long time. If you think = we should still change it, then I can open an issue for it.

On Thu, Mar 3, 2016 at 12:2= 0 PM, Stephan Ewen <sewen@apache.org> wrote:
Okay, that is a change from the original= behavior, introduced in HA. Originally, if the connection attempts failed,= it always returned the=C2=A0InetAddress.getLocalHost() interface.
I think= we should change it back to that, because that interface is by far the bes= t possible heuristic.

<= div class=3D"gmail_quote">On Thu, Mar 3, 2016 at 11:39 AM, Till Rohrmann <trohrmann@apache.org> wrote:

If I= =E2=80=99m not mistaken, then it=E2=80=99s not necessarily true that the he= uristic returns InetAddress.getLocalHost() in all c= ases. The heuristic will select the first network interface with the afore-= mentioned conditions but before returning it, it will try a last time to co= nnect to the JM via the interface bound to InetAddress.get= LocalHost(). However, if this fails, then the heuristically selected= network interface will be returned.

=E2=80=8B

On Thu, Mar 3, 2016 at 10:49 AM, Stephan Ewen <sew= en@apache.org> wrote:
If the ThasManager cannot connect to the JobManager, it will us= e the interface that is bound to the machine's host name ("InetAdd= ress.getLocalHost()").

So, the best way to fix this= would be to make sure that all machines have a proper network configuratio= n. Then Flink would either use an address that can connect (via trying vari= ous interfaces), or it would default back to the hostname/interface that is= configured on the machine.


On Thu, Mar 3, 2016 at 10:4= 3 AM, Till Rohrmann <trohrmann@apache.org> wrote:

Hi Max,

th= e problem is that before starting the TM, we have to find the network inter= face which is reachable by the other machines. So what we do is to connect = to the current JobManager. If it should happen, as in your case, that the J= obManager just died and the new JM address has not been written to ZooKeepe= r, then the TMs don=E2=80=99t have much choice other than using the heurist= ic.

I can=E2=80=99t really t= ell why eth1 is chosen over eth0. The condition is that the interface addre= ss is an Inet4Address, no link local address as wel= l as not a loopback address.

Thus, Ufuk=E2=80=99s solution, to increase akka.lookup.ti= meout seems to be the easiest solution to solve your problem. I=E2= =80=99ve checked the default value is set to 10 s w= hich might be a bit too low for restarting a new JM and publishing its addr= ess via ZooKeeper.

Cheers,Till

=E2=80=8B

On Thu, Mar 3, 2016 at 10:28 AM, Ufuk Celebi <<= a href=3D"mailto:uce@apache.org" target=3D"_blank">uce@apache.org> wrote:
I had an offline chat with T= ill about this. He pointed out that the
address is chosen once at start up time (while not being able to
connect to the old job manager) and then it stays fixed at eth1.

You can increase the lookup timeout by setting akka.lookup.timeout to
a higher value (like 100 s). This is the only workaroud I'm aware of at this point. Maybe Till can chime in here whether this has other
implications as well?

=E2=80=93 Ufuk

On Thu, Mar 3, 2016 at 9:59 AM, Ufuk Celebi <uce@apache.org> wrote:
> Hey Max!
>
> for the first WARN in
> org.apache.flink.runtime.webmonitor.JobManagerRetriever: this is
> expected if the new leader has not updated ZooKeeper yet. The
> important thing is that the new leading job manager is eventually
> retrieved. This did happen, right?
>
> Regarding eth1 vs. eth0: After the new job manager becomes leader, the=
> task manager should re-try connecting to it with the same strategy as<= br> > in the initial connection establishment (e.g. try SLOW first and only<= br> > fall back to HEURISTIC). Can you see in the logs whether this happens?=
>
> The best thing would be to share the complete logs. Is this possible?<= br> > If not publicly, feel free to send it to me privately (uce at apache > org).
>
> =E2=80=93 Ufuk
>
>
> On Thu, Mar 3, 2016 at 9:21 AM, Maximilian Bode
> <m= aximilian.bode@tngtech.com> wrote:
>> Hi everyone,
>>
>> we are trying to get to work JobManager HA in the context of a per= -job YARN
>> session using the 1.0.0-rc3 from a few days ago and are having a p= roblem
>> concerning task managers with several network interfaces.
>>
>> After manually killing the job manager process, the jobmanager.log= on the
>> newly allocated second job manager reads:
>> ---
>> 2016-03-02 18:01:09,635 WARN=C2=A0 Remoting
>> - Tried to associate with unreachable remote address
>> [akka.tcp://flink@10.127.68.136:34811]. Address is now= gated for 5000 ms,
>> all messages to this address will be delivered to dead letters. Re= ason:
>> Connection refused: /10.127.68.136:34811
>> 2016-03-02 18:01:09,644 WARN
>> org.apache.flink.runtime.webmonitor.JobManagerRetriever=C2=A0 =C2= =A0 =C2=A0 =C2=A0- Failed to
>> retrieve leader gateway and port.
>> akka.actor.ActorNotFound: Actor not found for:
>> ActorSelection[Anchor(akka.tcp://flink@10.127.68.136:34811= /),
>> Path(/user/jobmanager)]
>> at
>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelecti= on.scala:65)
>> at
>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelecti= on.scala:63)
>> at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32) >> at
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1= (BatchingExecutor.scala:67)
>> at
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(B= atchingExecutor.scala:82)
>> at
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(Batching= Executor.scala:59)
>> at
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(Batching= Executor.scala:59)
>> at scala.concurrent.BlockContext$.withBlockContext(BlockContext.sc= ala:72)
>> at akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala= :58)
>> at
>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatc= hedExecute(Future.scala:74)
>> at akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.s= cala:110)
>> at
>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execut= e(Future.scala:73)
>> at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise= .scala:40)
>> at
>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.s= cala:248)
>> at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:267)
>> at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:508)=
>> at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:541)=
>> at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:531)
>> at
>> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(= RemoteActorRefProvider.scala:87)
>> at akka.remote.EndpointWriter.postStop(Endpoint.scala:561)
>> at akka.actor.Actor$class.aroundPostStop(Actor.scala:475)
>> at akka.remote.EndpointActor.aroundPostStop(Endpoint.scala:415) >> at
>> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHan= dling$$finishTerminate(FaultHandling.scala:210)
>> at akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.= scala:172)
>> at akka.actor.ActorCell.terminate(ActorCell.scala:369)
>> at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:462)
>> at akka.actor.ActorCell.systemInvoke(ActorCell.scala:478)
>> at akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:27= 9)
>> at akka.dispatch.Mailbox.run(Mailbox.scala:220)
>> at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
>> at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java= :260)
>> at
>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinP= ool.java:1339)
>> at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.j= ava:1979)
>> at
>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerT= hread.java:107)
>> ---
>> where the IP not found is from the old job manager. So far, is thi= s the
>> expected behavior?
>>
>> The problem then arises on a new task manager, which also tries to= connect
>> to the old job manager unsuccessfully. The ZooKeeperLeaderRetrieva= lService
>> starts cycling through the available network interfaces, as can be= seen in
>> the relevant taskmanager.log:
>> ---
>> 2016-03-02 18:01:13,636 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalS= ervice=C2=A0 -
>> Starting ZooKeeperLeaderRetrievalService.
>> 2016-03-02 18:01:13,646 INFO
>> org.apache.flink.runtime.util.LeaderRetrievalUtils=C2=A0 =C2=A0 = =C2=A0 =C2=A0 =C2=A0 =C2=A0 - Trying to
>> select the network interface and address to use by connecting to t= he leading
>> JobManager.
>> 2016-03-02 18:01:13,646 INFO
>> org.apache.flink.runtime.util.LeaderRetrievalUtils=C2=A0 =C2=A0 = =C2=A0 =C2=A0 =C2=A0 =C2=A0 - TaskManager
>> will try to connect for 10000 milliseconds before falling back to = heuristics
>> 2016-03-02 18:01:13,712 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Retrieved new target address /10.127.68.136:34811.
>> 2016-03-02 18:01:14,079 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Trying to connect to address /10.127.68.136:34811
>> 2016-03-02 18:01:14,082 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address
>> 'task.manager.eth0.hostname.com/10.1= 27.68.136': Connection refused
>> 2016-03-02 18:01:14,082 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/10.127.68.136': Connect= ion refused
>> 2016-03-02 18:01:14,082 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/10.120.193.110': Conne= ction refused
>> 2016-03-02 18:01:14,082 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/10.127.68.136': Connect= ion refused
>> 2016-03-02 18:01:14,083 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/127.0.0.1': Connection refu= sed
>> 2016-03-02 18:01:14,083 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/10.120.193.110': Conne= ction refused
>> 2016-03-02 18:01:14,083 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/10.127.68.136': Connect= ion refused
>> 2016-03-02 18:01:14,083 INFO=C2=A0 org.apache.flink.runtime.net.Co= nnectionUtils
>> - Failed to connect from address '/127.0.0.1': Connection refu= sed
>> ---
>> After five repetitions, the task manager stops trying to retrieve = the leader
>> and using the HEURISTIC strategy ends up using=C2=A0 eth1 (10.120.= 193.110) from
>> now on:
>> ---
>> 2016-03-02 18:01:23,650 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalS= ervice=C2=A0 -
>> Stopping ZooKeeperLeaderRetrievalService.
>> 2016-03-02 18:01:23,655 INFO=C2=A0 org.apache.zookeeper.ClientCnxn=
>> - EventThread shut down
>> 2016-03-02 18:01:23,655 INFO=C2=A0 org.apache.zookeeper.ZooKeeper<= br> >> - Session: 0x25229757cff035b closed
>> 2016-03-02 18:01:23,664 INFO
>> org.apache.flink.runtime.taskmanager.TaskManager=C2=A0 =C2=A0 =C2= =A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 - TaskManager
>> will use hostname/address 'task.manager.eth1.host= name.com' (10.120.193.110)
>> for communication.
>> ---
>> Following the new jobmanager is discovered and the taskmanager is = able to
>> register at the jobmanager using eth1. The problem is that connect= ions TO
>> eth1 are not possible. So flink should always use eth0. The except= ion we
>> later see is:
>> ---
>> java.io.IOException: Connecting the channel failed: Connecting to = remote
>> task manager + 'other.task.manager.eth1.hostname/10.120.193= .111:46620' has
>> failed. This might indicate that the remote task manager has been = lost.
>> at
>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFa= ctory$ConnectingChannel.waitForChannel(PartitionRequestClientFactory.java:1= 96)
>> at
>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFa= ctory$ConnectingChannel.access$000(PartitionRequestClientFactory.java:131)<= br> >> at
>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientFa= ctory.createPartitionRequestClient(PartitionRequestClientFactory.java:83) >> at
>> org.apache.flink.runtime.io.network.netty.NettyConnectionManager.c= reatePartitionRequestClient(NettyConnectionManager.java:60)
>> at
>> org.apache.flink.runtime.io.network.partition.consumer.RemoteInput= Channel.requestSubpartition(RemoteInputChannel.java:115)
>> at
>> org.apache.flink.runtime.io.network.partition.consumer.SingleInput= Gate.requestPartitions(SingleInputGate.java:388)
>> at
>> org.apache.flink.runtime.io.network.partition.consumer.SingleInput= Gate.getNextBufferOrEvent(SingleInputGate.java:411)
>> at
>> org.apache.flink.streaming.runtime.io.BarrierBuffer.getNextNonBloc= ked(BarrierBuffer.java:108)
>> at
>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.process= Input(StreamInputProcessor.java:175)
>> at
>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(On= eInputStreamTask.java:65)
>> at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamT= ask.java:224)
>> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559) >> at java.lang.Thread.run(Thread.java:744)
>> ---
>> The root cause seems to be that network interface selection is sti= ll using
>> the old jobmanager location and hence is not able to choose the ri= ght
>> interface. In particular, it seems that iteration order over the n= etwork
>> interfaces differs between the HEURISTIC and SLOW strategy, which = then leads
>> to the wrong interface being selected.
>>
>> Cheers,
>>=C2=A0 Max
>> =E2=80=94
>> Maximilian Bode * Junior Consultant * maximilian.bode@tngtech.com
>> TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterf=C3=B6hr= ing
>> Gesch=C3=A4ftsf=C3=BChrer: Henrik Klagges, Christoph Stock, Dr. Ro= bert Dahlke
>> Sitz: Unterf=C3=B6hring * Amtsgericht M=C3=BCnchen * HRB 135082 >>







--001a114a91b2218462052d2507db--