[
https://issues.apache.org/jira/browse/FLINK-2821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15582974#comment-15582974
]
Philipp von dem Bussche commented on FLINK-2821:
------------------------------------------------
Thanks [~mxm], I am not getting this Exception anymore, however I don't think
this is working yet.
I have to admit though that I had to change my environment slightly in which I
am testing since I am currently travelling. I don't at the moment have access
to the Rancher environment so I am purely bringing up a Docker container on my
Mac within a (non-native) docker-machine which basically means I have a
virtualbox virtual machine running on my Mac which runs the Docker daemon and
from this virtual machine I am running my Docker containers at the moment. I do
believe though that this test environment is quite similar to my initial test
with Rancher. I have exposed port 6123 from the docker container to the host
(aka the virtual machine).
This happens on my non-customized 1.1.3 build (not the one you have created for
me):
I am trying to access my Flink's jobmanager rpc address (doing a simple flink
list from my Mac) like this:
PHILIPPs-MacBook:~ philipp$ flink list --jobmanager 192.168.99.100:6123 #
192.168.99.100 is the docker host's IP / the IP of the virtual machine
I am getting this error message after a while:
Retrieving JobManager.
Using address /192.168.99.100:6123 to connect to JobManager.
------------------------------------------------------------
The program finished with the following exception:
org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException: Could not
retrieve the leader gateway
at
org.apache.flink.runtime.util.LeaderRetrievalUtils.retrieveLeaderGateway(LeaderRetrievalUtils.java:127)
at
org.apache.flink.client.program.ClusterClient.getJobManagerGateway(ClusterClient.java:644)
at
org.apache.flink.client.CliFrontend.getJobManagerGateway(CliFrontend.java:868)
at org.apache.flink.client.CliFrontend.list(CliFrontend.java:387)
at
org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1008)
at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1048)
Caused by: java.util.concurrent.TimeoutException: Futures timed out after
[10000 milliseconds]
at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:219)
at
scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
at scala.concurrent.Await$$anonfun$result$1.apply(package.scala:190)
at
scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
at scala.concurrent.Await$.result(package.scala:190)
at scala.concurrent.Await.result(package.scala)
at
org.apache.flink.runtime.util.LeaderRetrievalUtils.retrieveLeaderGateway(LeaderRetrievalUtils.java:125)
... 5 more
And in my Flink's jobmanager log file I am seeing this error message:
2016-10-17 17:58:46,088 INFO org.apache.flink.runtime.jobmanager.JobManager
- Starting JobManager at
akka.tcp://[email protected]:6123/user/jobmanager.
2016-10-17 17:58:46,108 INFO
org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
- Trying to associate with JobManager leader
akka.tcp://[email protected]:6123/user/jobmanager
2016-10-17 17:58:46,132 INFO org.apache.flink.runtime.jobmanager.JobManager
- JobManager akka.tcp://[email protected]:6123/user/jobmanager was
granted leadership with leader session ID None.
2016-10-17 17:58:46,140 INFO
org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
- Resource Manager associating with leading JobManager
Actor[akka://flink/user/jobmanager#-1164381512] - leader session null
2016-10-17 17:59:34,896 ERROR akka.remote.EndpointWriter
- dropping message [class akka.actor.ActorSelectionMessage] for
non-local recipient [Actor[akka.tcp://[email protected]:6123/]] arriving at
[akka.tcp://[email protected]:6123] inbound addresses are
[akka.tcp://[email protected]:6123]
2016-10-17 17:59:45,052 WARN akka.remote.ReliableDeliverySupervisor
- Association with remote system
[akka.tcp://[email protected]:51492] has failed, address is now gated for
[5000] ms. Reason is: [Disassociated].
I would think that the difference between this and the Rancher approach would
be that Rancher introduces this third IP address (10.x) which gets used when
using the Rancher DNS name between containers in a Rancher environment.
Anyways when I am using the custom version that you have sent me and I
configure my jobmanager like this:
jobmanager.rpc.address: 192.168.99.100
jobmanager.rpc.bind-address: da54c7ceaaa9 # container's host name resolving to
the 172.x address
jobmanager.rpc.port: 6123
jobmanager.rpc.bind-port: 6123
The jobmanager startup fails with a message like this in the log:
2016-10-17 17:40:42,104 INFO org.apache.flink.runtime.jobmanager.JobManager
- Starting JobManager on 192.168.99.100:6123 with execution mode
CLUSTER
2016-10-17 17:40:42,169 INFO org.apache.flink.runtime.jobmanager.JobManager
- Security is not enabled. Starting non-authenticated JobManager.
2016-10-17 17:40:42,174 INFO org.apache.flink.util.NetUtils
- Unable to allocate on port 6123, due to error: Address not
available
2016-10-17 17:40:42,189 ERROR org.apache.flink.runtime.jobmanager.JobManager
- Failed to run JobManager.
java.lang.RuntimeException: Unable to do further retries starting the actor
system
at
org.apache.flink.runtime.jobmanager.JobManager$.retryOnBindException(JobManager.scala:2116)
at
org.apache.flink.runtime.jobmanager.JobManager$.runJobManager(JobManager.scala:2056)
at
org.apache.flink.runtime.jobmanager.JobManager$.main(JobManager.scala:1981)
at org.apache.flink.runtime.jobmanager.JobManager.main(JobManager.scala)
As I said I am travelling at the moment and will be back home Wednesday night
and I should be able to test more then.
Thanks
> Change Akka configuration to allow accessing actors from different URLs
> -----------------------------------------------------------------------
>
> Key: FLINK-2821
> URL: https://issues.apache.org/jira/browse/FLINK-2821
> Project: Flink
> Issue Type: Bug
> Components: Distributed Coordination
> Reporter: Robert Metzger
> Assignee: Maximilian Michels
>
> Akka expects the actor's URL to be exactly matching.
> As pointed out here, cases where users were complaining about this:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Error-trying-to-access-JM-through-proxy-td3018.html
> - Proxy routing (as described here, send to the proxy URL, receiver
> recognizes only original URL)
> - Using hostname / IP interchangeably does not work (we solved this by
> always putting IP addresses into URLs, never hostnames)
> - Binding to multiple interfaces (any local 0.0.0.0) does not work. Still
> no solution to that (but seems not too much of a restriction)
> I am aware that this is not possible due to Akka, so it is actually not a
> Flink bug. But I think we should track the resolution of the issue here
> anyways because its affecting our user's satisfaction.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)