The port has its separate configuration parameter jobmanager.rpc.port [1]

[1]
https://ci.apache.org/projects/flink/flink-docs-master/docs/deployment/config/#jobmanager-rpc-port-1

On Wed, Sep 29, 2021 at 10:11 AM Javier Vegas <jve...@strava.com> wrote:

> Matthias, thanks for the suggestion! I changed my jobmanager.rpc.address
> param from $HOSTNAME to $HOST:$PORT0 which in the log I see resolves
> properly to the host IP and port mapped to 8081
>
> 2021-09-29 07:58:05.452 [main] INFO
>  org.apache.flink.runtime.entrypoint.ClusterEntrypoint  -
> -Djobmanager.rpc.address=10.0.22.114:31894
>
> which is very promising. But sadly a little bit later appmaster dies with
> this errror:
>
> 2021-09-29 07:58:05.648 [main] INFO
>  org.apache.flink.runtime.entrypoint.ClusterEntrypoint  - Initializing
> cluster services.
> 2021-09-29 07:58:05.674 [main] INFO
>  org.apache.flink.runtime.entrypoint.ClusterEntrypoint  - Shutting
> MesosSessionClusterEntrypoint down with application status FAILED.
> Diagnostics org.apache.flink.configurati
> on.IllegalConfigurationException: The configured hostname is not valid
> at
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:179)
> at
> org.apache.flink.util.NetUtils.unresolvedHostAndPortToNormalizedString(NetUtils.java:197)
> at
> org.apache.flink.runtime.clusterframework.BootstrapTools.startRemoteActorSystem(BootstrapTools.java:207)
> at
> org.apache.flink.runtime.clusterframework.BootstrapTools.startRemoteActorSystem(BootstrapTools.java:152)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils$AkkaRpcServiceBuilder.createAndStart(AkkaRpcServiceUtils.java:370)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils$AkkaRpcServiceBuilder.createAndStart(AkkaRpcServiceUtils.java:344)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.createRemoteRpcService(AkkaRpcServiceUtils.java:92)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:294)
> at
> org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint.initializeServices(MesosSessionClusterEntrypoint.java:61)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:239)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$1(ClusterEntrypoint.java:189)
> at java.base/java.security.AccessController.doPrivileged(Native Method)
> at java.base/javax.security.auth.Subject.doAs(Unknown Source)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
> at
> org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:186)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:600)
> at
> org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint.main(MesosSessionClusterEntrypoint.java:114)
> Caused by: java.lang.IllegalArgumentException
> at
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:122)
> at
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:177)
> ... 17 more
> .
> 2021-09-29 07:58:05.685 [main] ERROR
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint  - Could not start
> cluster entrypoint MesosSessionClusterEntrypoint.
> org.apache.flink.runtime.entrypoint.ClusterEntrypointException: Failed to
> initialize the cluster entrypoint MesosSessionClusterEntrypoint.
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:212)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:600)
> at
> org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint.main(MesosSessionClusterEntrypoint.java:114)
> Caused by: org.apache.flink.configuration.IllegalConfigurationException:
> The configured hostname is not valid
> at
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:179)
> at
> org.apache.flink.util.NetUtils.unresolvedHostAndPortToNormalizedString(NetUtils.java:197)
> at
> org.apache.flink.runtime.clusterframework.BootstrapTools.startRemoteActorSystem(BootstrapTools.java:207)
> at
> org.apache.flink.runtime.clusterframework.BootstrapTools.startRemoteActorSystem(BootstrapTools.java:152)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils$AkkaRpcServiceBuilder.createAndStart(AkkaRpcServiceUtils.java:370)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils$AkkaRpcServiceBuilder.createAndStart(AkkaRpcServiceUtils.java:344)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.createRemoteRpcService(AkkaRpcServiceUtils.java:92)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:294)
> at
> org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint.initializeServices(MesosSessionClusterEntrypoint.java:61)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:239)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$1(ClusterEntrypoint.java:189)
> at java.base/java.security.AccessController.doPrivileged(Native Method)
> at java.base/javax.security.auth.Subject.doAs(Unknown Source)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
> at
> org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:186)
> ... 2 common frames omitted
> Caused by: java.lang.IllegalArgumentException: null
> at
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:122)
> at
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:177)
> ... 17 common frames omitted
>
>
>
> On Wed, Sep 29, 2021 at 12:16 AM Matthias Pohl <matth...@ververica.com>
> wrote:
>
>> One thing that was puzzling me yesterday when reading your post: Have you
>> tried $HOST instead of $HOSTNAME in the Marathon configuration? When I
>> played around with Mesos, I remember using HOST to resolve the host's IP
>> address instead of the host's name. It could be that the hostname itself
>> cannot be resolved to the right IP address. But I struggled to find proper
>> documentation to back that up. Only in the recipes section of the Marathon
>> docs [1], HOST was used as well.
>>
>> Matthias
>>
>> [1]
>> https://mesosphere.github.io/marathon/docs/recipes.html#command-executor-health-checks
>>
>> On Wed, Sep 29, 2021 at 3:37 AM Javier Vegas <jve...@strava.com> wrote:
>>
>>> Another update:  Looking more carefully in my appmaster log, I see the
>>> following
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -
>>> Registering as new framework.
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -
>>> -----------------------------------------------------------------------------
>>>
>>> ---
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -  Mesos
>>> Info:
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Master
>>> URL: 10.0.18.246:5050
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -  Framework
>>> Info:
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     ID:
>>> (none)
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Name:
>>> flink-test
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Failover
>>> Timeout (secs): 604800.0
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Role:
>>> *
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     
>>> Capabilities:
>>> (none)
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     
>>> Principal:
>>> (none)
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Host:
>>> 311dcf7fd77c
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -     Web
>>> UI: http://311dcf7fd77c:8081
>>>
>>> 2021-09-29 01:15:39.680 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  -
>>> -----------------------------------------------------------------------------
>>>
>>> ---
>>>
>>>
>>> which is picking up the mesos.master and
>>> mesos.resourcemanager.framework.name params I am passing to
>>> mesos-appmaster.sh
>>>
>>>
>>> In my Mesos dashboard I can see the framework has been created with the
>>> right name, but has no associated agents/tasks to it. So at least Flink has
>>> been able to connect to the Mesos master to create the framework
>>>
>>>
>>> Later in the mesos-appmaster log is when I see the Mesos connection
>>> errors:
>>>
>>>
>>> 2021-09-29 01:15:39.726 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> o.a.f.r.resourcemanager.slotmanager.DeclarativeSlotManager  - Starting
>>> the slot manager.
>>>
>>> 2021-09-29 01:15:39.815 [flink-akka.actor.default-dispatcher-2] DEBUG
>>> org.apache.flink.mesos.scheduler.ConnectionMonitor  - State change
>>> (StoppedState -> StoppedState) with data ()
>>>
>>> 2021-09-29 01:15:39.823 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> o.a.f.runtime.resourcemanager.active.ActiveResourceManager  - Trigger
>>> heartbeat request.
>>>
>>> 2021-09-29 01:15:39.823 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> org.apache.flink.mesos.scheduler.ReconciliationCoordinator  - State
>>> change (Suspended -> Suspended) with data ReconciliationData(Map(),0)
>>>
>>> 2021-09-29 01:15:39.823 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> o.a.f.runtime.resourcemanager.active.ActiveResourceManager  - Trigger
>>> heartbeat request.
>>>
>>> 2021-09-29 01:15:39.824 [flink-akka.actor.default-dispatcher-3] INFO
>>> org.apache.flink.mesos.scheduler.ConnectionMonitor  - Connecting to
>>> Mesos...
>>>
>>> 2021-09-29 01:15:39.825 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> org.apache.flink.mesos.scheduler.ConnectionMonitor  - State change
>>> (StoppedState -> ConnectingState) with data ()
>>>
>>> 2021-09-29 01:15:39.826 [flink-akka.actor.default-dispatcher-3] INFO
>>> o.a.f.m.runtime.clusterframework.MesosResourceManagerDriver  - Mesos
>>> resource manager started.
>>>
>>> 2021-09-29 01:15:39.831 [flink-akka.actor.default-dispatcher-4] DEBUG
>>> org.apache.flink.mesos.scheduler.LaunchCoordinator  - State change
>>> (Suspended -> Suspended) with data GatherData(List(),List())
>>>
>>> 2021-09-29 01:15:44.843 [flink-akka.actor.default-dispatcher-4] WARN
>>> org.apache.flink.mesos.scheduler.ConnectionMonitor  - Unable to connect
>>> to Mesos; still trying...
>>>
>>> 2021-09-29 01:15:49.843 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> o.a.f.runtime.resourcemanager.active.ActiveResourceManager  - Trigger
>>> heartbeat request.
>>>
>>> 2021-09-29 01:15:49.844 [flink-akka.actor.default-dispatcher-3] DEBUG
>>> o.a.f.runtime.resourcemanager.active.ActiveResourceManager  - Trigger
>>> heartbeat request.
>>>
>>>
>>>
>>>
>>> So why the appmaster was able to connect to Mesos master to create the
>>> framework but failed to connect later to do whatever it does later?
>>>
>>>
>>> One possible issue I see is that the framework is set with web UI in h
>>> ttp://311dcf7fd77c:8081 which can not be resolved from the Mesos
>>> master. 311dcf7fd77c is the result of doing hostname on the Docker
>>> container, and the Mesos master can not resolve that name. I could try to
>>> replace the Docker container hostname with the Docker host hostname, but
>>> the host port that gets mapped to 8081 on the container is a random port
>>> that I can not know beforehand. Does Mesos master try to reach Flink using
>>> that Web UI setting? Could this be the issue causing my connection problem,
>>> or is this a red herring and the problem is a different one?
>>>
>>>
>>> Thanks,
>>>
>>>
>>> Javier Vegas
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Tue, Sep 28, 2021 at 10:23 AM Javier Vegas <jve...@strava.com> wrote:
>>>
>>>> Thanks, Matthias!
>>>>
>>>> There are lots of apps deployed to the Mesos cluster, the task manager
>>>> itself is deployed to Mesos via Marathon.  In the Mesos log I can see the
>>>> Job manager agent starting, but no error messages related to it. As you
>>>> say, TaskManagers don't even have the chance to get confused about
>>>> variables, since the Job Manager can not connect to the Mesos master to
>>>> tell it to start the Task Managers.
>>>>
>>>> Thanks,
>>>>
>>>> Javier
>>>>
>>>> On Tue, Sep 28, 2021 at 7:59 AM Matthias Pohl <matth...@ververica.com>
>>>> wrote:
>>>>
>>>>> Hi Javier,
>>>>> I don't see anything that's configured in the wrong way based on the
>>>>> jobmanager logs you've provided. Have you been able to deploy other
>>>>> applications to this Mesos cluster? Do the Mesos master logs reveal
>>>>> anything? The variable resolution on the TaskManager side is a valid
>>>>> concern shared by Roman since it's easy to run into such an issue. But the
>>>>> JobManager logs indicate that the JobManager is not able to contact the
>>>>> Mesos master. Hence, I'd assume that it's not related to the TaskManagers
>>>>> not coming up.
>>>>>
>>>>> Best,
>>>>> Matthias
>>>>>
>>>>> On Tue, Sep 28, 2021 at 2:45 PM Roman Khachatryan <ro...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> No additional ports need to be open as far as I know.
>>>>>>
>>>>>> Probably, $HOSTNAME is substituted for something not resolvable on
>>>>>> TMs?
>>>>>>
>>>>>> Please also make sure that the following gets executed before
>>>>>> mesos-appmaster.sh:
>>>>>> export HADOOP_CLASSPATH=$(hadoop classpath)
>>>>>> export MESOS_NATIVE_JAVA_LIBRARY=/path/to/lib/libmesos.so
>>>>>> (as per the documentation you linked)
>>>>>>
>>>>>> Regards,
>>>>>> Roman
>>>>>>
>>>>>> On Mon, Sep 27, 2021 at 7:38 PM Javier Vegas <jve...@strava.com>
>>>>>> wrote:
>>>>>> >
>>>>>> > I am trying to start Flink 1.13.2 on Mesos following the
>>>>>> instrucions in
>>>>>> https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/deployment/resource-providers/mesos/
>>>>>> and using Marathon to deploy a Docker image with both the Flink and my
>>>>>> binaries.
>>>>>> >
>>>>>> > My entrypoint for the Docker image is:
>>>>>> >
>>>>>> >
>>>>>> > /opt/flink/bin/mesos-appmaster.sh \
>>>>>> >
>>>>>> >       -Djobmanager.rpc.address=$HOSTNAME \
>>>>>> >
>>>>>> >       -Dmesos.resourcemanager.framework.user=flink \
>>>>>> >
>>>>>> >       -Dmesos.master=10.0.18.246:5050 \
>>>>>> >
>>>>>> >       -Dmesos.resourcemanager.tasks.cpus=6
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> > When mesos-appmaster.sh starts, in the stderr I see this:
>>>>>> >
>>>>>> >
>>>>>> > I0927 16:50:32.306691 801308 exec.cpp:164] Version: 1.7.3
>>>>>> >
>>>>>> > I0927 16:50:32.310277 801345 exec.cpp:238] Executor registered on
>>>>>> agent f671d9ee-57f6-4f92-b1b2-3137676f6cdf-S6090
>>>>>> >
>>>>>> > I0927 16:50:32.311120 801355 executor.cpp:130] Registered docker
>>>>>> executor on 10.0.20.177
>>>>>> >
>>>>>> > I0927 16:50:32.311394 801345 executor.cpp:186] Starting task
>>>>>> tl_flink_prod.fb215c64-1fb2-11ec-9ce6-aaa2e9cb6ba0
>>>>>> >
>>>>>> > WARNING: Your kernel does not support swap limit capabilities or
>>>>>> the cgroup is not mounted. Memory limited without swap.
>>>>>> >
>>>>>> > WARNING: An illegal reflective access operation has occurred
>>>>>> >
>>>>>> > WARNING: Illegal reflective access by
>>>>>> org.apache.hadoop.security.authentication.util.KerberosUtil
>>>>>> (file:/opt/flink/lib/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar) to method
>>>>>> sun.security.krb5.Config.getInstance()
>>>>>> >
>>>>>> > WARNING: Please consider reporting this to the maintainers of
>>>>>> org.apache.hadoop.security.authentication.util.KerberosUtil
>>>>>> >
>>>>>> > WARNING: Use --illegal-access=warn to enable warnings of further
>>>>>> illegal reflective access operations
>>>>>> >
>>>>>> > WARNING: All illegal access operations will be denied in a future
>>>>>> release
>>>>>> >
>>>>>> > I0927 16:50:43.622053   237 sched.cpp:232] Version: 1.7.3
>>>>>> >
>>>>>> > I0927 16:50:43.624439   328 sched.cpp:336] New master detected at
>>>>>> master@10.0.18.246:5050
>>>>>> >
>>>>>> > I0927 16:50:43.624779   328 sched.cpp:356] No credentials provided.
>>>>>> Attempting to register without authentication
>>>>>> >
>>>>>> >
>>>>>> > where the "New master detected" line is promising.
>>>>>> >
>>>>>> > However, on the Flink UI I see only the jobmanager started, and
>>>>>> there are no task managers.  Getting into the Docker container, I see 
>>>>>> this
>>>>>> in the log:
>>>>>> >
>>>>>> > WARN  org.apache.flink.mesos.scheduler.ConnectionMonitor  - Unable
>>>>>> to connect to Mesos; still trying...
>>>>>> >
>>>>>> >
>>>>>> > I have verified that from the container I can access the Mesos
>>>>>> container 10.0.18.246:5050
>>>>>> >
>>>>>> >
>>>>>> > Does any other port besides the web UI port 5050 need to be open
>>>>>> for mesos-appmaster to connect with the Mesos master?
>>>>>> >
>>>>>> >
>>>>>> > In the appmaster log (attached) I see one exception that I don't
>>>>>> know if they are related to the Mesos connection problem, one is
>>>>>> >
>>>>>> >
>>>>>> > java.io.FileNotFoundException: HADOOP_HOME and hadoop.home.dir are
>>>>>> unset.
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.util.Shell.checkHadoopHomeInner(Shell.java:448)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.util.Shell.checkHadoopHome(Shell.java:419)
>>>>>> >
>>>>>> >         at org.apache.hadoop.util.Shell.<clinit>(Shell.java:496)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:79)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1555)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.SecurityUtil.getLogSlowLookupsEnabled(SecurityUtil.java:497)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.SecurityUtil.<clinit>(SecurityUtil.java:90)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:289)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:277)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:833)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:803)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:676)
>>>>>> >
>>>>>> >         at
>>>>>> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native
>>>>>> Method)
>>>>>> >
>>>>>> >         at
>>>>>> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(Unknown
>>>>>> Source)
>>>>>> >
>>>>>> >         at
>>>>>> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(Unknown
>>>>>> Source)
>>>>>> >
>>>>>> >         at java.base/java.lang.reflect.Method.invoke(Unknown Source)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.flink.runtime.util.EnvironmentInformation.getHadoopUser(EnvironmentInformation.java:215)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.flink.runtime.util.EnvironmentInformation.logEnvironmentInfo(EnvironmentInformation.java:432)
>>>>>> >
>>>>>> >         at
>>>>>> org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint.main(MesosSessionClusterEntrypoint.java:95)
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> > I am not trying (yet) to run in high availability mode, so I am not
>>>>>> sure if I need to have HADOOP_HOME set or not, but I don't see anything
>>>>>> about HADOOP_HOME in the FLink docs.
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> > Any tips on how I can fix my Docker+Marathon+Mesos environment so
>>>>>> Flink can connect to my Mesos master?
>>>>>> >
>>>>>> >
>>>>>> > Thanks,
>>>>>> >
>>>>>> >
>>>>>> > Javier Vegas
>>>>>> >
>>>>>> >
>>>>>
>>>>>
>>

Reply via email to