I see, thanks for the info. I only have access to my cluster via SLURM and
we don't have ssh between our nodes which is why I haven't really
considered the Standalone mode. A colleague has set up YARN on SLURM and it
was just the easiest to use. I briefly looked into the Flink Standalone
mode but dropped it because I thought YARN would be possible after all. It
seems I'm going to have a deeper look into starting the master and slaves
with SLURM's srun instead of ssh (I guess a slight modification of
start-cluster.sh should do the job).

On Thu, Oct 1, 2015 at 11:30 AM, Robert Metzger <rmetz...@apache.org> wrote:

> Hi,
> there is currently no option for forcing certain containers onto specific
> machines.
> For running the JM (or any other YARN container) on the AM host, you first
> need to have a NodeManager running on the host with the RM. Maybe YARN is
> smart enough to schedule the small JM container onto that machine.
>
> I don't know your exact setup, but maybe it would make sense for you to
> run Flink in the standalone cluster mode instead with YARN. It seems that
> you have a very good idea how and where you want to run the Flink services
> in your cluster. YARN is designed to be an abstraction between the cluster
> and the application, that's why its a bit difficult to schedule the
> containers to specific machines.
>
> Robert
>
>
>
> On Thu, Oct 1, 2015 at 11:24 AM, Robert Schmidtke <ro.schmid...@gmail.com>
> wrote:
>
>> Hi Robert,
>>
>> I had a job failure yesterday with what I believe is the setup I have
>> described above. However when trying to reproduce now, the behavior is the
>> same: Flink waiting for resources to become available. So no hard error.
>>
>> Ok, the looping makes sense then. I haven't thought about shared setups.
>> I'm still figuring out how all parameters play together, i.e. -yn, -yjm,
>> -ytm and the memory limits in yarn-site.xml. This will need some testing
>> and I'll come back with a proper description once I think I know what's
>> going on.
>>
>> When running Flink on YARN, is it easily possible to place the Flink JM
>> where the YARN Resource Manager sits, and all the TMs with the remaining
>> Node Managers?
>>
>> Robert
>>
>> On Thu, Oct 1, 2015 at 10:53 AM, Robert Metzger <rmetz...@apache.org>
>> wrote:
>>
>>> Hi,
>>>
>>> It is interesting to note that when I set both 
>>> yarn.nodemanager.resource.memory-mb
>>>> and yarn.scheduler.maximum-allocation-mb to 56G I get a proper error
>>>> when requesting 56G and 1M, but when setting 
>>>> yarn.nodemanager.resource.memory-mb
>>>> to 56G and yarn.scheduler.maximum-allocation-mb to 54G I don't get an
>>>> error but the aforementioned endless loop.
>>>
>>>
>>> is it a "hard error" (failing) you're getting or just "WARN" log
>>> messages. I'm asking because I've added some code some time ago to do some
>>> checks before deploying Flink on YARN. These checks will print WARN log
>>> messages if the requested YARN session/job does not fit onto the cluster.
>>> This "endless loop" exists because in many production environments Flink
>>> can just wait for resources to become available, for example when other
>>> containers are finishing.
>>>
>>>
>>> Robert
>>>
>>> On Wed, Sep 30, 2015 at 6:33 PM, Robert Schmidtke <
>>> ro.schmid...@gmail.com> wrote:
>>>
>>>> Hi Robert,
>>>>
>>>> thanks for your reply. It got me digging into my setup and I discovered
>>>> that one TM was scheduled next to the JM. When specifying -yn 7 the
>>>> documentation suggests that this is the number of TMs (of which I wanted
>>>> 7), and I thought an additional container would be used for the JM (my YARN
>>>> cluster has 8 containers). Anyway with this setup the memory added up to
>>>> 56G and 1M (40G per TM and 16G 1M for the JM), but I set a hard maximum of
>>>> 56G in my yarn-site.xml which is why the request could not be fulfilled. It
>>>> is interesting to note that when I set
>>>> both yarn.nodemanager.resource.memory-mb
>>>> and yarn.scheduler.maximum-allocation-mb to 56G I get a proper error when
>>>> requesting 56G and 1M, but when setting yarn.nodemanager.resource.memory-mb
>>>> to 56G and yarn.scheduler.maximum-allocation-mb to 54G I don't get an error
>>>> but the aforementioned endless loop. Note I
>>>> have yarn.nodemanager.vmem-check-enabled set to false. This is probably a
>>>> YARN issue then / my bad configuration.
>>>>
>>>> I'm in a rush now (to get to the Flink meetup) and thus will check the
>>>> documentation later to see how to deploy the TMs and JM on separate
>>>> machines each, since that is not what's happening at the moment, but this
>>>> is what I'd like to have. Thanks again and see you in an hour.
>>>>
>>>> Cheers
>>>> Robert
>>>>
>>>> On Wed, Sep 30, 2015 at 5:19 PM, Robert Metzger <rmetz...@apache.org>
>>>> wrote:
>>>>
>>>>> Hi Robert,
>>>>>
>>>>> the problem here is that YARN's scheduler (there are different
>>>>> schedulers in YARN: FIFO, CapacityScheduler, ...) is not giving Flink's
>>>>> ApplicationMaster/JobManager all the containers it is requesting. By
>>>>> increasing the size of the AM/JM container, there is probably no memory
>>>>> left to fit the last TaskManager container.
>>>>> I also experienced this issue, when I wanted to run a Flink job on
>>>>> YARN and the containers were fitting theoretically, but YARN was not 
>>>>> giving
>>>>> me all the containers I requested.
>>>>> Back then, I asked on the yarn-dev list [1] (there were also some
>>>>> off-list emails) but we could not resolve the issue.
>>>>>
>>>>> Can you check the resource manager logs? Maybe there is a log message
>>>>> which explains why the container request of Flink's AM is not fulfilled.
>>>>>
>>>>>
>>>>> [1]
>>>>> http://search-hadoop.com/m/AsBtCilK5r1pKLjf1&subj=Re+QUESTION+Allocating+a+full+YARN+cluster
>>>>>
>>>>> On Wed, Sep 30, 2015 at 5:02 PM, Robert Schmidtke <
>>>>> ro.schmid...@gmail.com> wrote:
>>>>>
>>>>>> It's me again. This is a strange issue, I hope I managed to find the
>>>>>> right keywords. I got 8 machines, 1 for the JM, the other 7 are TMs with
>>>>>> 64G of memory each.
>>>>>>
>>>>>> When running my job like so:
>>>>>>
>>>>>> $FLINK_HOME/bin/flink run -m yarn-cluster -yjm 16384 -ytm 40960 -yn 7
>>>>>> .....
>>>>>>
>>>>>> The job completes without any problems. When running it like so:
>>>>>>
>>>>>> $FLINK_HOME/bin/flink run -m yarn-cluster -yjm 16385 -ytm 40960 -yn 7
>>>>>> .....
>>>>>>
>>>>>> (note the one more M of memory for the JM), the execution stalls,
>>>>>> continuously reporting:
>>>>>>
>>>>>> .....
>>>>>> TaskManager status (6/7)
>>>>>> TaskManager status (6/7)
>>>>>> TaskManager status (6/7)
>>>>>> .....
>>>>>>
>>>>>> I did some poking around, but I couldn't find any direct correlation
>>>>>> with the code.
>>>>>>
>>>>>> The JM log says:
>>>>>>
>>>>>> .....
>>>>>> 16:49:01,893 INFO  org.apache.flink.yarn.ApplicationMaster$
>>>>>>            -  JVM Options:
>>>>>> 16:49:01,893 INFO  org.apache.flink.yarn.ApplicationMaster$
>>>>>>            -     -Xmx12289M
>>>>>> .....
>>>>>>
>>>>>> but then continues to report
>>>>>>
>>>>>> .....
>>>>>> 16:52:59,311 INFO
>>>>>>  org.apache.flink.yarn.ApplicationMaster$$anonfun$2$$anon$1    - The user
>>>>>> requested 7 containers, 6 running. 1 containers missing
>>>>>> 16:52:59,831 INFO
>>>>>>  org.apache.flink.yarn.ApplicationMaster$$anonfun$2$$anon$1    - The user
>>>>>> requested 7 containers, 6 running. 1 containers missing
>>>>>> 16:53:00,351 INFO
>>>>>>  org.apache.flink.yarn.ApplicationMaster$$anonfun$2$$anon$1    - The user
>>>>>> requested 7 containers, 6 running. 1 containers missing
>>>>>> .....
>>>>>>
>>>>>> forever until I cancel the job.
>>>>>>
>>>>>> If you have any ideas I'm happy to try them out. Thanks in advance
>>>>>> for any hints! Cheers.
>>>>>>
>>>>>> Robert
>>>>>> --
>>>>>> My GPG Key ID: 336E2680
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> My GPG Key ID: 336E2680
>>>>
>>>
>>>
>>
>>
>> --
>> My GPG Key ID: 336E2680
>>
>
>


-- 
My GPG Key ID: 336E2680

Reply via email to