Hi Ovidiu,

good choice on your research topic ;)

I think doing some hands on experiments will help you to understand much
better how Flink works and what you can do with it.

If I got it right:
> -with standalone (cluster) you can run multiple workloads if you have
> enough resources, else the job will be rejected.
> -with a yarn session, yarn will accept the job but will only execute it
> when there are enough resources.


That's not right. The YARN session and standalone cluster mode are
basically the same.
Both the YARN session and the cluster mode will run job in parallel if
there are not enough resources and they both will reject jobs if not enough
resources are there.


My point on *scheduling*:
> If I have an installation (Flink over Yarn for example) and in my cluster
> I have enough resources to serve multiple requests.
> Some jobs are running permanently, some are not. I want to be able to
> schedule jobs concurrently. My options right now, if I understand
> correctly, is to either wait for the current job to finish (assuming it has
> acquired all the available resources) or to stop the current job, in case I
> have other jobs with higher priorities. This could be related also to the
> resource elasticity you mentioned.


Yes, resource elasticity in Flink will mitigate such issues. We would be
able to respond to YARN's preemption requests if jobs with higher
priorities are requesting additional resources.

On Fri, Nov 20, 2015 at 2:07 PM, Ovidiu-Cristian MARCU <
ovidiu-cristian.ma...@inria.fr> wrote:

> Thank you, Robert!
>
> My research interest includes Flink (I am a PhD student, BigStorage EU
> project, Inria Rennes) so I am currently preparing some experiments in
> order to understand better how it works.
>
> If I got it right:
> -with standalone (cluster) you can run multiple workloads if you have
> enough resources, else the job will be rejected.
> -with a yarn session, yarn will accept the job but will only execute it
> when there are enough resources.
>
> My point on *scheduling*:
> If I have an installation (Flink over Yarn for example) and in my cluster
> I have enough resources to serve multiple requests.
> Some jobs are running permanently, some are not. I want to be able to
> schedule jobs concurrently. My options right now, if I understand
> correctly, is to either wait for the current job to finish (assuming it has
> acquired all the available resources) or to stop the current job, in case I
> have other jobs with higher priorities. This could be related also to the
> resource elasticity you mentioned.
>
> Best regards,
> Ovidiu
>
> On 20 Nov 2015, at 13:34, Robert Metzger <rmetz...@apache.org> wrote:
>
> Hi,
> I'll fix the link in the YARN documentation. Thank you for reporting the
> issue.
>
> I'm not aware of any discussions or implementations related to the
> scheduling. From my experience working with users and also from the mailing
> list, I don't think that such features are very important.
> Since streaming jobs usually run permanently, there is no need to queue
> jobs somehow.
> For batch jobs, YARN is taking care of the resource allocation (in
> practice this means that the job has to wait until the required resources
> are available).
>
> There are some discussions (and user requests) regarding resource
> elasticity going on and I think we'll add features for dynamically changing
> the size of a Flink cluster on YARN while a job is running.
>
> Which features are you missing wrt to scheduling in Flink? Please let me
> know if there is anything blocking you from using Flink in production and
> we'll see what we can do.
>
> Regards,
> Robert
>
>
>
> On Fri, Nov 20, 2015 at 1:24 PM, Ovidiu-Cristian MARCU <
> ovidiu-cristian.ma...@inria.fr> wrote:
>
>> Hi,
>>
>> The link to FAQ (
>> https://ci.apache.org/projects/flink/flink-docs-release-0.10/faq.html) is
>> on the yarn setup 0.10 documentation page (
>> https://ci.apache.org/projects/flink/flink-docs-release-0.10/setup/yarn_setup.html)
>> described in this sentence: *If you have troubles using the Flink YARN
>> client, have a look in the FAQ section
>> <https://ci.apache.org/projects/flink/flink-docs-release-0.10/faq.html>.*
>>
>> Is the scheduling features considered for next releases?
>>
>> Thank you.
>> Best regards,
>> Ovidiu
>>
>> On 20 Nov 2015, at 11:59, Robert Metzger <rmetz...@apache.org> wrote:
>>
>> Hi Ovidiu,
>>
>> you can submit multiple programs to a running Flink cluster (or a YARN
>> session). Flink does currently not have any queuing mechanism.
>> The JobManager will reject a program if there are not enough free
>> resources for it. If there are enough resources for multiple programs,
>> they'll run concurrently.
>> Note that Flink is not starting separate JVMs for the programs, so if
>> one program is doing a System.exit(0), it is killing the entire JVM,
>> including other running programs.
>>
>> You can start as many YARN sessions (or single jobs to YARN) as you have
>> resources available on the cluster. The resource allocation is up to the
>> scheduler you've configured in YARN.
>>
>> In general, we recommend to start a YARN session per program. You can
>> also directly submit a Flink program to YARN.
>>
>> Where did you find the link to the FAQ? The link on the front page is
>> working: http://flink.apache.org/faq.html
>>
>>
>>
>> On Fri, Nov 20, 2015 at 11:41 AM, Ovidiu-Cristian MARCU <
>> ovidiu-cristian.ma...@inria.fr> wrote:
>>
>>> Hi,
>>>
>>> I am currently interested in experimenting on Flink over Hadoop YARN.
>>> I am documenting from the documentation we have here:
>>> https://ci.apache.org/projects/flink/flink-docs-release-0.10/setup/yarn_setup.html
>>>
>>> There is a subsection *Start Flink Session* which states the following: *A
>>> session will start all required Flink services (JobManager and
>>> TaskManagers) so that you can submit programs to the cluster. Note that you
>>> can run multiple programs per session.*
>>>
>>> Can you be more precise regarding the multiple programs per session? If
>>> I submit multiple programs concurently what will happen (can I?)? Maybe
>>> they will run in a FIFO fashion or what should I expect?
>>>
>>> The internals section specify that users can execute multiple Flink Yarn
>>> sessions in parallel. This is great, this invites to static partitioning of
>>> resources in order to run multiple applications concurrently. Do you
>>> support a fair scheduler similar to what Spark claims it has?
>>>
>>> There is FAQ section (
>>> https://ci.apache.org/projects/flink/flink-docs-release-0.10/faq.html)
>>> resource that is missing, can this be updated?
>>>
>>> Thank you.
>>>
>>> Best regards,
>>> Ovidiu
>>>
>>>
>>
>>
>>
>
>

Reply via email to