Re: Mesos + Spark users going forward?

2021-04-07 Thread dmcwhorter
We are using the mesos integration at Premier (https://www.premierinc.com/). 
Obviously with the move to the attic we will likely move away from Mesos in
the future.  I think deprecating the mesos integration makes sense.  We
would probably continue to utilize the spark mesos components for another
release or two if its possible to include them for a little bit longer
before they're removed.



--
Sent from: http://apache-spark-user-list.1001560.n3.nabble.com/

-
To unsubscribe e-mail: user-unsubscr...@spark.apache.org



Re: Mesos + Spark users going forward?

2021-04-07 Thread Mridul Muralidharan
Unfortunate about Mesos, +1 on deprecation of mesos integration.

Regards,
Mridul


On Wed, Apr 7, 2021 at 7:12 AM Sean Owen  wrote:

> I noted that Apache Mesos is moving to the attic, so won't be actively
> developed soon:
>
> https://lists.apache.org/thread.html/rab2a820507f7c846e54a847398ab20f47698ec5bce0c8e182bfe51ba%40%3Cdev.mesos.apache.org%3E
>
> That doesn't mean people will stop using it as a Spark resource manager
> soon. But it suggests the Spark + Mesos integration is a candidate for
> deprecation and eventual removal in Spark at some point.
>
> This is mostly an informal poll: are there Mesos users out there planning
> to continue using it for a while? moving off? just seeing if it's
> reasonable to even start deprecation in 3.2.0.
>
> Sean
>
>


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-26 Thread Chawla,Sumit
What is the expected effect of reducing the mesosExecutor.cores to zero?
What functionality of executor is impacted? Is the impact is just that it
just behaves like a regular process?

Regards
Sumit Chawla


On Mon, Dec 26, 2016 at 9:25 AM, Michael Gummelt <mgumm...@mesosphere.io>
wrote:

> > Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
> allocation
>
> Maybe for CPU, but definitely not for memory.  Executors never shut down
> in fine-grained mode, which means you only elastically grow and shrink CPU
> usage, not memory.
>
> On Sat, Dec 24, 2016 at 10:14 PM, Davies Liu <davies@gmail.com> wrote:
>
>> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
>> allocation, but have to pay a little more overhead for launching a
>> task, which should be OK if the task is not trivial.
>>
>> Since the direct result (up to 1M by default) will also go through
>> mesos, it's better to tune it lower, otherwise mesos could become the
>> bottleneck.
>>
>> spark.task.maxDirectResultSize
>>
>> On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>> wrote:
>> > Tim,
>> >
>> > We will try to run the application in coarse grain mode, and share the
>> > findings with you.
>> >
>> > Regards
>> > Sumit Chawla
>> >
>> >
>> > On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com>
>> wrote:
>> >
>> >> Dynamic allocation works with Coarse grain mode only, we wasn't aware
>> >> a need for Fine grain mode after we enabled dynamic allocation support
>> >> on the coarse grain mode.
>> >>
>> >> What's the reason you're running fine grain mode instead of coarse
>> >> grain + dynamic allocation?
>> >>
>> >> Tim
>> >>
>> >> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
>> >> <mehdi.mezi...@ldmobile.net> wrote:
>> >> > We will be interested by the results if you give a try to Dynamic
>> >> allocation
>> >> > with mesos !
>> >> >
>> >> >
>> >> > - Mail Original -
>> >> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
>> >> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
>> >> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
>> >> > <user@spark.apache.org>, d...@spark.apache.org
>> >> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam /
>> Berlin /
>> >> > Berne / Rome / Stockholm / Vienne
>> >> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
>> >> >
>> >> >
>> >> >> Is this problem of idle executors sticking around solved in Dynamic
>> >> >> Resource Allocation?  Is there some timeout after which Idle
>> executors
>> >> can
>> >> >> just shutdown and cleanup its resources.
>> >> >
>> >> > Yes, that's exactly what dynamic allocation does.  But again I have
>> no
>> >> idea
>> >> > what the state of dynamic allocation + mesos is.
>> >> >
>> >> > On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit <
>> sumitkcha...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Great.  Makes much better sense now.  What will be reason to have
>> >> >> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
>> >> include
>> >> >> the number of cores for tasks.
>> >> >>
>> >> >> So in my case it seems like 30 CPUs are allocated to executors.  And
>> >> there
>> >> >> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of
>> 30 is
>> >> >> maintained till the last task exits.  This explains the gap.
>>  Thanks
>> >> >> everyone.  I am still not sure how this number 30 is calculated.  (
>> Is
>> >> it
>> >> >> dynamic based on current resources, or is it some configuration.  I
>> >> have 32
>> >> >> nodes in my cluster).
>> >> >>
>> >> >> Is this problem of idle executors sticking around solved in Dynamic
>> >> >> Resource Allocation?  Is there some timeout after which Idle
>> executors
>> >> can
>> >> >> just shutdown and cleanup its resources.
>> >>

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-26 Thread Jacek Laskowski
Thanks a LOT, Michael!

Pozdrawiam,
Jacek Laskowski

https://medium.com/@jaceklaskowski/
Mastering Apache Spark 2.0 https://bit.ly/mastering-apache-spark
Follow me at https://twitter.com/jaceklaskowski


On Mon, Dec 26, 2016 at 10:04 PM, Michael Gummelt
<mgumm...@mesosphere.io> wrote:
> In fine-grained mode (which is deprecated), Spark tasks (which are threads)
> were implemented as Mesos tasks.  When a Mesos task starts and stops, its
> underlying cgroup, and therefore the resources its consuming on the cluster,
> grows or shrinks based on the resources allocated to the tasks, which in
> Spark is just CPU.  This is what I mean by CPU usage "elastically growing".
>
> However, all Mesos tasks are run by an "executor", which has its own
> resource allocation.  In Spark, the executor is the JVM, and all memory is
> allocated to the executor, because JVMs can't relinquish memory.  If memory
> were allocated to the tasks, then the cgroup's memory allocation would
> shrink when the task terminated, but the JVM's memory consumption would stay
> constant, and the JVM would OOM.
>
> And, without dynamic allocation, executors never terminate during the
> duration of a Spark job, because even if they're idle (no tasks), they still
> may be hosting shuffle files.  That's why dynamic allocation depends on an
> external shuffle service.  Since executors never terminate, and all memory
> is allocated to the executors, Spark jobs even in fine-grained mode only
> grow in memory allocation, they don't shrink.
>
> On Mon, Dec 26, 2016 at 12:39 PM, Jacek Laskowski <ja...@japila.pl> wrote:
>>
>> Hi Michael,
>>
>> That caught my attention...
>>
>> Could you please elaborate on "elastically grow and shrink CPU usage"
>> and how it really works under the covers? It seems that CPU usage is
>> just a "label" for an executor on Mesos. Where's this in the code?
>>
>> Pozdrawiam,
>> Jacek Laskowski
>> 
>> https://medium.com/@jaceklaskowski/
>> Mastering Apache Spark 2.0 https://bit.ly/mastering-apache-spark
>> Follow me at https://twitter.com/jaceklaskowski
>>
>>
>> On Mon, Dec 26, 2016 at 6:25 PM, Michael Gummelt <mgumm...@mesosphere.io>
>> wrote:
>> >> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
>> >> allocation
>> >
>> > Maybe for CPU, but definitely not for memory.  Executors never shut down
>> > in
>> > fine-grained mode, which means you only elastically grow and shrink CPU
>> > usage, not memory.
>> >
>> > On Sat, Dec 24, 2016 at 10:14 PM, Davies Liu <davies@gmail.com>
>> > wrote:
>> >>
>> >> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
>> >> allocation, but have to pay a little more overhead for launching a
>> >> task, which should be OK if the task is not trivial.
>> >>
>> >> Since the direct result (up to 1M by default) will also go through
>> >> mesos, it's better to tune it lower, otherwise mesos could become the
>> >> bottleneck.
>> >>
>> >> spark.task.maxDirectResultSize
>> >>
>> >> On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>> >> wrote:
>> >> > Tim,
>> >> >
>> >> > We will try to run the application in coarse grain mode, and share
>> >> > the
>> >> > findings with you.
>> >> >
>> >> > Regards
>> >> > Sumit Chawla
>> >> >
>> >> >
>> >> > On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com>
>> >> > wrote:
>> >> >
>> >> >> Dynamic allocation works with Coarse grain mode only, we wasn't
>> >> >> aware
>> >> >> a need for Fine grain mode after we enabled dynamic allocation
>> >> >> support
>> >> >> on the coarse grain mode.
>> >> >>
>> >> >> What's the reason you're running fine grain mode instead of coarse
>> >> >> grain + dynamic allocation?
>> >> >>
>> >> >> Tim
>> >> >>
>> >> >> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
>> >> >> <mehdi.mezi...@ldmobile.net> wrote:
>> >> >> > We will be interested by the results if you give a try to Dynamic
>> >> >> allocation
>> >> >> > with mesos !
>> >> >> >
>> >&g

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-26 Thread Michael Gummelt
In fine-grained mode (which is deprecated), Spark tasks (which are threads)
were implemented as Mesos tasks.  When a Mesos task starts and stops, its
underlying cgroup, and therefore the resources its consuming on the
cluster, grows or shrinks based on the resources allocated to the tasks,
which in Spark is just CPU.  This is what I mean by CPU usage "elastically
growing".

However, all Mesos tasks are run by an "executor", which has its own
resource allocation.  In Spark, the executor is the JVM, and all memory is
allocated to the executor, because JVMs can't relinquish memory.  If memory
were allocated to the tasks, then the cgroup's memory allocation would
shrink when the task terminated, but the JVM's memory consumption would
stay constant, and the JVM would OOM.

And, without dynamic allocation, executors never terminate during the
duration of a Spark job, because even if they're idle (no tasks), they
still may be hosting shuffle files.  That's why dynamic allocation depends
on an external shuffle service.  Since executors never terminate, and all
memory is allocated to the executors, Spark jobs even in fine-grained mode
only grow in memory allocation, they don't shrink.

On Mon, Dec 26, 2016 at 12:39 PM, Jacek Laskowski <ja...@japila.pl> wrote:

> Hi Michael,
>
> That caught my attention...
>
> Could you please elaborate on "elastically grow and shrink CPU usage"
> and how it really works under the covers? It seems that CPU usage is
> just a "label" for an executor on Mesos. Where's this in the code?
>
> Pozdrawiam,
> Jacek Laskowski
> 
> https://medium.com/@jaceklaskowski/
> Mastering Apache Spark 2.0 https://bit.ly/mastering-apache-spark
> Follow me at https://twitter.com/jaceklaskowski
>
>
> On Mon, Dec 26, 2016 at 6:25 PM, Michael Gummelt <mgumm...@mesosphere.io>
> wrote:
> >> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
> >> allocation
> >
> > Maybe for CPU, but definitely not for memory.  Executors never shut down
> in
> > fine-grained mode, which means you only elastically grow and shrink CPU
> > usage, not memory.
> >
> > On Sat, Dec 24, 2016 at 10:14 PM, Davies Liu <davies@gmail.com>
> wrote:
> >>
> >> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
> >> allocation, but have to pay a little more overhead for launching a
> >> task, which should be OK if the task is not trivial.
> >>
> >> Since the direct result (up to 1M by default) will also go through
> >> mesos, it's better to tune it lower, otherwise mesos could become the
> >> bottleneck.
> >>
> >> spark.task.maxDirectResultSize
> >>
> >> On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com>
> >> wrote:
> >> > Tim,
> >> >
> >> > We will try to run the application in coarse grain mode, and share the
> >> > findings with you.
> >> >
> >> > Regards
> >> > Sumit Chawla
> >> >
> >> >
> >> > On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com>
> wrote:
> >> >
> >> >> Dynamic allocation works with Coarse grain mode only, we wasn't aware
> >> >> a need for Fine grain mode after we enabled dynamic allocation
> support
> >> >> on the coarse grain mode.
> >> >>
> >> >> What's the reason you're running fine grain mode instead of coarse
> >> >> grain + dynamic allocation?
> >> >>
> >> >> Tim
> >> >>
> >> >> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
> >> >> <mehdi.mezi...@ldmobile.net> wrote:
> >> >> > We will be interested by the results if you give a try to Dynamic
> >> >> allocation
> >> >> > with mesos !
> >> >> >
> >> >> >
> >> >> > - Mail Original -
> >> >> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
> >> >> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
> >> >> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
> >> >> > <user@spark.apache.org>, d...@spark.apache.org
> >> >> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam /
> Berlin
> >> >> > /
> >> >> > Berne / Rome / Stockholm / Vienne
> >> >> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
> >> >> >
> >> >> >
> &g

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-26 Thread Jacek Laskowski
Hi Michael,

That caught my attention...

Could you please elaborate on "elastically grow and shrink CPU usage"
and how it really works under the covers? It seems that CPU usage is
just a "label" for an executor on Mesos. Where's this in the code?

Pozdrawiam,
Jacek Laskowski

https://medium.com/@jaceklaskowski/
Mastering Apache Spark 2.0 https://bit.ly/mastering-apache-spark
Follow me at https://twitter.com/jaceklaskowski


On Mon, Dec 26, 2016 at 6:25 PM, Michael Gummelt <mgumm...@mesosphere.io> wrote:
>> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
>> allocation
>
> Maybe for CPU, but definitely not for memory.  Executors never shut down in
> fine-grained mode, which means you only elastically grow and shrink CPU
> usage, not memory.
>
> On Sat, Dec 24, 2016 at 10:14 PM, Davies Liu <davies@gmail.com> wrote:
>>
>> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
>> allocation, but have to pay a little more overhead for launching a
>> task, which should be OK if the task is not trivial.
>>
>> Since the direct result (up to 1M by default) will also go through
>> mesos, it's better to tune it lower, otherwise mesos could become the
>> bottleneck.
>>
>> spark.task.maxDirectResultSize
>>
>> On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>> wrote:
>> > Tim,
>> >
>> > We will try to run the application in coarse grain mode, and share the
>> > findings with you.
>> >
>> > Regards
>> > Sumit Chawla
>> >
>> >
>> > On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com> wrote:
>> >
>> >> Dynamic allocation works with Coarse grain mode only, we wasn't aware
>> >> a need for Fine grain mode after we enabled dynamic allocation support
>> >> on the coarse grain mode.
>> >>
>> >> What's the reason you're running fine grain mode instead of coarse
>> >> grain + dynamic allocation?
>> >>
>> >> Tim
>> >>
>> >> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
>> >> <mehdi.mezi...@ldmobile.net> wrote:
>> >> > We will be interested by the results if you give a try to Dynamic
>> >> allocation
>> >> > with mesos !
>> >> >
>> >> >
>> >> > - Mail Original -
>> >> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
>> >> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
>> >> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
>> >> > <user@spark.apache.org>, d...@spark.apache.org
>> >> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin
>> >> > /
>> >> > Berne / Rome / Stockholm / Vienne
>> >> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
>> >> >
>> >> >
>> >> >> Is this problem of idle executors sticking around solved in Dynamic
>> >> >> Resource Allocation?  Is there some timeout after which Idle
>> >> >> executors
>> >> can
>> >> >> just shutdown and cleanup its resources.
>> >> >
>> >> > Yes, that's exactly what dynamic allocation does.  But again I have
>> >> > no
>> >> idea
>> >> > what the state of dynamic allocation + mesos is.
>> >> >
>> >> > On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit
>> >> > <sumitkcha...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Great.  Makes much better sense now.  What will be reason to have
>> >> >> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
>> >> include
>> >> >> the number of cores for tasks.
>> >> >>
>> >> >> So in my case it seems like 30 CPUs are allocated to executors.  And
>> >> there
>> >> >> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of
>> >> >> 30 is
>> >> >> maintained till the last task exits.  This explains the gap.
>> >> >> Thanks
>> >> >> everyone.  I am still not sure how this number 30 is calculated.  (
>> >> >> Is
>> >> it
>> >> >> dynamic based on current resources, or is it some configuration.  I
>> >> have 32
>> >> >> nodes in my cluster

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-26 Thread Michael Gummelt
> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
allocation

Maybe for CPU, but definitely not for memory.  Executors never shut down in
fine-grained mode, which means you only elastically grow and shrink CPU
usage, not memory.

On Sat, Dec 24, 2016 at 10:14 PM, Davies Liu <davies@gmail.com> wrote:

> Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
> allocation, but have to pay a little more overhead for launching a
> task, which should be OK if the task is not trivial.
>
> Since the direct result (up to 1M by default) will also go through
> mesos, it's better to tune it lower, otherwise mesos could become the
> bottleneck.
>
> spark.task.maxDirectResultSize
>
> On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com>
> wrote:
> > Tim,
> >
> > We will try to run the application in coarse grain mode, and share the
> > findings with you.
> >
> > Regards
> > Sumit Chawla
> >
> >
> > On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com> wrote:
> >
> >> Dynamic allocation works with Coarse grain mode only, we wasn't aware
> >> a need for Fine grain mode after we enabled dynamic allocation support
> >> on the coarse grain mode.
> >>
> >> What's the reason you're running fine grain mode instead of coarse
> >> grain + dynamic allocation?
> >>
> >> Tim
> >>
> >> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
> >> <mehdi.mezi...@ldmobile.net> wrote:
> >> > We will be interested by the results if you give a try to Dynamic
> >> allocation
> >> > with mesos !
> >> >
> >> >
> >> > - Mail Original -
> >> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
> >> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
> >> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
> >> > <user@spark.apache.org>, d...@spark.apache.org
> >> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin
> /
> >> > Berne / Rome / Stockholm / Vienne
> >> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
> >> >
> >> >
> >> >> Is this problem of idle executors sticking around solved in Dynamic
> >> >> Resource Allocation?  Is there some timeout after which Idle
> executors
> >> can
> >> >> just shutdown and cleanup its resources.
> >> >
> >> > Yes, that's exactly what dynamic allocation does.  But again I have no
> >> idea
> >> > what the state of dynamic allocation + mesos is.
> >> >
> >> > On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit <sumitkcha...@gmail.com
> >
> >> > wrote:
> >> >>
> >> >> Great.  Makes much better sense now.  What will be reason to have
> >> >> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
> >> include
> >> >> the number of cores for tasks.
> >> >>
> >> >> So in my case it seems like 30 CPUs are allocated to executors.  And
> >> there
> >> >> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of
> 30 is
> >> >> maintained till the last task exits.  This explains the gap.   Thanks
> >> >> everyone.  I am still not sure how this number 30 is calculated.  (
> Is
> >> it
> >> >> dynamic based on current resources, or is it some configuration.  I
> >> have 32
> >> >> nodes in my cluster).
> >> >>
> >> >> Is this problem of idle executors sticking around solved in Dynamic
> >> >> Resource Allocation?  Is there some timeout after which Idle
> executors
> >> can
> >> >> just shutdown and cleanup its resources.
> >> >>
> >> >>
> >> >> Regards
> >> >> Sumit Chawla
> >> >>
> >> >>
> >> >> On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt <
> >> mgumm...@mesosphere.io>
> >> >> wrote:
> >> >>>
> >> >>> >  I should preassume that No of executors should be less than
> number
> >> of
> >> >>> > tasks.
> >> >>>
> >> >>> No.  Each executor runs 0 or more tasks.
> >> >>>
> >> >>> Each executor consumes 1 CPU, and each task running on that executor
> >>

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-24 Thread Davies Liu
Using 0 for spark.mesos.mesosExecutor.cores is better than dynamic
allocation, but have to pay a little more overhead for launching a
task, which should be OK if the task is not trivial.

Since the direct result (up to 1M by default) will also go through
mesos, it's better to tune it lower, otherwise mesos could become the
bottleneck.

spark.task.maxDirectResultSize

On Mon, Dec 19, 2016 at 3:23 PM, Chawla,Sumit <sumitkcha...@gmail.com> wrote:
> Tim,
>
> We will try to run the application in coarse grain mode, and share the
> findings with you.
>
> Regards
> Sumit Chawla
>
>
> On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com> wrote:
>
>> Dynamic allocation works with Coarse grain mode only, we wasn't aware
>> a need for Fine grain mode after we enabled dynamic allocation support
>> on the coarse grain mode.
>>
>> What's the reason you're running fine grain mode instead of coarse
>> grain + dynamic allocation?
>>
>> Tim
>>
>> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
>> <mehdi.mezi...@ldmobile.net> wrote:
>> > We will be interested by the results if you give a try to Dynamic
>> allocation
>> > with mesos !
>> >
>> >
>> > - Mail Original -
>> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
>> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
>> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
>> > <user@spark.apache.org>, d...@spark.apache.org
>> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin /
>> > Berne / Rome / Stockholm / Vienne
>> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
>> >
>> >
>> >> Is this problem of idle executors sticking around solved in Dynamic
>> >> Resource Allocation?  Is there some timeout after which Idle executors
>> can
>> >> just shutdown and cleanup its resources.
>> >
>> > Yes, that's exactly what dynamic allocation does.  But again I have no
>> idea
>> > what the state of dynamic allocation + mesos is.
>> >
>> > On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>> > wrote:
>> >>
>> >> Great.  Makes much better sense now.  What will be reason to have
>> >> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
>> include
>> >> the number of cores for tasks.
>> >>
>> >> So in my case it seems like 30 CPUs are allocated to executors.  And
>> there
>> >> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of 30 is
>> >> maintained till the last task exits.  This explains the gap.   Thanks
>> >> everyone.  I am still not sure how this number 30 is calculated.  ( Is
>> it
>> >> dynamic based on current resources, or is it some configuration.  I
>> have 32
>> >> nodes in my cluster).
>> >>
>> >> Is this problem of idle executors sticking around solved in Dynamic
>> >> Resource Allocation?  Is there some timeout after which Idle executors
>> can
>> >> just shutdown and cleanup its resources.
>> >>
>> >>
>> >> Regards
>> >> Sumit Chawla
>> >>
>> >>
>> >> On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt <
>> mgumm...@mesosphere.io>
>> >> wrote:
>> >>>
>> >>> >  I should preassume that No of executors should be less than number
>> of
>> >>> > tasks.
>> >>>
>> >>> No.  Each executor runs 0 or more tasks.
>> >>>
>> >>> Each executor consumes 1 CPU, and each task running on that executor
>> >>> consumes another CPU.  You can customize this via
>> >>> spark.mesos.mesosExecutor.cores
>> >>> (https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md)
>> and
>> >>> spark.task.cpus
>> >>> (https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md)
>> >>>
>> >>> On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit <sumitkcha...@gmail.com
>> >
>> >>> wrote:
>> >>>>
>> >>>> Ah thanks. looks like i skipped reading this "Neither will executors
>> >>>> terminate when they’re idle."
>> >>>>
>> >>>> So in my job scenario,  I should preassume that No of executors should
>> >>>> be less than n

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Chawla,Sumit
Tim,

We will try to run the application in coarse grain mode, and share the
findings with you.

Regards
Sumit Chawla


On Mon, Dec 19, 2016 at 3:11 PM, Timothy Chen <tnac...@gmail.com> wrote:

> Dynamic allocation works with Coarse grain mode only, we wasn't aware
> a need for Fine grain mode after we enabled dynamic allocation support
> on the coarse grain mode.
>
> What's the reason you're running fine grain mode instead of coarse
> grain + dynamic allocation?
>
> Tim
>
> On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
> <mehdi.mezi...@ldmobile.net> wrote:
> > We will be interested by the results if you give a try to Dynamic
> allocation
> > with mesos !
> >
> >
> > - Mail Original -
> > De: "Michael Gummelt" <mgumm...@mesosphere.io>
> > À: "Sumit Chawla" <sumitkcha...@gmail.com>
> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
> > <user@spark.apache.org>, d...@spark.apache.org
> > Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin /
> > Berne / Rome / Stockholm / Vienne
> > Objet: Re: Mesos Spark Fine Grained Execution - CPU count
> >
> >
> >> Is this problem of idle executors sticking around solved in Dynamic
> >> Resource Allocation?  Is there some timeout after which Idle executors
> can
> >> just shutdown and cleanup its resources.
> >
> > Yes, that's exactly what dynamic allocation does.  But again I have no
> idea
> > what the state of dynamic allocation + mesos is.
> >
> > On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit <sumitkcha...@gmail.com>
> > wrote:
> >>
> >> Great.  Makes much better sense now.  What will be reason to have
> >> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
> include
> >> the number of cores for tasks.
> >>
> >> So in my case it seems like 30 CPUs are allocated to executors.  And
> there
> >> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of 30 is
> >> maintained till the last task exits.  This explains the gap.   Thanks
> >> everyone.  I am still not sure how this number 30 is calculated.  ( Is
> it
> >> dynamic based on current resources, or is it some configuration.  I
> have 32
> >> nodes in my cluster).
> >>
> >> Is this problem of idle executors sticking around solved in Dynamic
> >> Resource Allocation?  Is there some timeout after which Idle executors
> can
> >> just shutdown and cleanup its resources.
> >>
> >>
> >> Regards
> >> Sumit Chawla
> >>
> >>
> >> On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt <
> mgumm...@mesosphere.io>
> >> wrote:
> >>>
> >>> >  I should preassume that No of executors should be less than number
> of
> >>> > tasks.
> >>>
> >>> No.  Each executor runs 0 or more tasks.
> >>>
> >>> Each executor consumes 1 CPU, and each task running on that executor
> >>> consumes another CPU.  You can customize this via
> >>> spark.mesos.mesosExecutor.cores
> >>> (https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md)
> and
> >>> spark.task.cpus
> >>> (https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md)
> >>>
> >>> On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit <sumitkcha...@gmail.com
> >
> >>> wrote:
> >>>>
> >>>> Ah thanks. looks like i skipped reading this "Neither will executors
> >>>> terminate when they’re idle."
> >>>>
> >>>> So in my job scenario,  I should preassume that No of executors should
> >>>> be less than number of tasks. Ideally one executor should execute 1
> or more
> >>>> tasks.  But i am observing something strange instead.  I start my job
> with
> >>>> 48 partitions for a spark job. In mesos ui i see that number of tasks
> is 48,
> >>>> but no. of CPUs is 78 which is way more than 48.  Here i am assuming
> that 1
> >>>> CPU is 1 executor.   I am not specifying any configuration to set
> number of
> >>>> cores per executor.
> >>>>
> >>>> Regards
> >>>> Sumit Chawla
> >>>>
> >>>>
> >>>> On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere
> >>>> <jo...@mesosphere.io> wrote:
> >>>>>
> >>>>> That makes sense. From the docum

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Timothy Chen
Dynamic allocation works with Coarse grain mode only, we wasn't aware
a need for Fine grain mode after we enabled dynamic allocation support
on the coarse grain mode.

What's the reason you're running fine grain mode instead of coarse
grain + dynamic allocation?

Tim

On Mon, Dec 19, 2016 at 2:45 PM, Mehdi Meziane
<mehdi.mezi...@ldmobile.net> wrote:
> We will be interested by the results if you give a try to Dynamic allocation
> with mesos !
>
>
> - Mail Original -
> De: "Michael Gummelt" <mgumm...@mesosphere.io>
> À: "Sumit Chawla" <sumitkcha...@gmail.com>
> Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
> <user@spark.apache.org>, d...@spark.apache.org
> Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin /
> Berne / Rome / Stockholm / Vienne
> Objet: Re: Mesos Spark Fine Grained Execution - CPU count
>
>
>> Is this problem of idle executors sticking around solved in Dynamic
>> Resource Allocation?  Is there some timeout after which Idle executors can
>> just shutdown and cleanup its resources.
>
> Yes, that's exactly what dynamic allocation does.  But again I have no idea
> what the state of dynamic allocation + mesos is.
>
> On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit <sumitkcha...@gmail.com>
> wrote:
>>
>> Great.  Makes much better sense now.  What will be reason to have
>> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't include
>> the number of cores for tasks.
>>
>> So in my case it seems like 30 CPUs are allocated to executors.  And there
>> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of 30 is
>> maintained till the last task exits.  This explains the gap.   Thanks
>> everyone.  I am still not sure how this number 30 is calculated.  ( Is it
>> dynamic based on current resources, or is it some configuration.  I have 32
>> nodes in my cluster).
>>
>> Is this problem of idle executors sticking around solved in Dynamic
>> Resource Allocation?  Is there some timeout after which Idle executors can
>> just shutdown and cleanup its resources.
>>
>>
>> Regards
>> Sumit Chawla
>>
>>
>> On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt <mgumm...@mesosphere.io>
>> wrote:
>>>
>>> >  I should preassume that No of executors should be less than number of
>>> > tasks.
>>>
>>> No.  Each executor runs 0 or more tasks.
>>>
>>> Each executor consumes 1 CPU, and each task running on that executor
>>> consumes another CPU.  You can customize this via
>>> spark.mesos.mesosExecutor.cores
>>> (https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md) and
>>> spark.task.cpus
>>> (https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md)
>>>
>>> On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>>> wrote:
>>>>
>>>> Ah thanks. looks like i skipped reading this "Neither will executors
>>>> terminate when they’re idle."
>>>>
>>>> So in my job scenario,  I should preassume that No of executors should
>>>> be less than number of tasks. Ideally one executor should execute 1 or more
>>>> tasks.  But i am observing something strange instead.  I start my job with
>>>> 48 partitions for a spark job. In mesos ui i see that number of tasks is 
>>>> 48,
>>>> but no. of CPUs is 78 which is way more than 48.  Here i am assuming that 1
>>>> CPU is 1 executor.   I am not specifying any configuration to set number of
>>>> cores per executor.
>>>>
>>>> Regards
>>>> Sumit Chawla
>>>>
>>>>
>>>> On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere
>>>> <jo...@mesosphere.io> wrote:
>>>>>
>>>>> That makes sense. From the documentation it looks like the executors
>>>>> are not supposed to terminate:
>>>>>
>>>>> http://spark.apache.org/docs/latest/running-on-mesos.html#fine-grained-deprecated
>>>>>>
>>>>>> Note that while Spark tasks in fine-grained will relinquish cores as
>>>>>> they terminate, they will not relinquish memory, as the JVM does not give
>>>>>> memory back to the Operating System. Neither will executors terminate 
>>>>>> when
>>>>>> they’re idle.
>>>>>
>>>>>
>>>>> I suppose your task to executor CPU ratio is low enough that it l

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Mehdi Meziane
We will be interested by the results if you give a try to Dynamic allocation 
with mesos ! 



- Mail Original - 
De: "Michael Gummelt" <mgumm...@mesosphere.io> 
À: "Sumit Chawla" <sumitkcha...@gmail.com> 
Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User" 
<user@spark.apache.org>, d...@spark.apache.org 
Envoyé: Lundi 19 Décembre 2016 22h42:55 GMT +01:00 Amsterdam / Berlin / Berne / 
Rome / Stockholm / Vienne 
Objet: Re: Mesos Spark Fine Grained Execution - CPU count 



> Is this problem of idle executors sticking around solved in Dynamic Resource 
> Allocation? Is there some timeout after which Idle executors can just 
> shutdown and cleanup its resources. 

Yes, that's exactly what dynamic allocation does. But again I have no idea what 
the state of dynamic allocation + mesos is. 



On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit < sumitkcha...@gmail.com > wrote: 



Great. Makes much better sense now. What will be reason to have 
spark.mesos.mesosExecutor. cores more than 1, as this number doesn't include 
the number of cores for tasks. 


So in my case it seems like 30 CPUs are allocated to executors. And there are 
48 tasks so 48 + 30 = 78 CPUs. And i am noticing this gap of 30 is maintained 
till the last task exits. This explains the gap. Thanks everyone. I am still 
not sure how this number 30 is calculated. ( Is it dynamic based on current 
resources, or is it some configuration. I have 32 nodes in my cluster). 


Is this problem of idle executors sticking around solved in Dynamic Resource 
Allocation? Is there some timeout after which Idle executors can just shutdown 
and cleanup its resources. 





Regards 
Sumit Chawla 





On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt < mgumm...@mesosphere.io > 
wrote: 





> I should preassume that No of executors should be less than number of tasks. 

No. Each executor runs 0 or more tasks. 

Each executor consumes 1 CPU, and each task running on that executor consumes 
another CPU. You can customize this via spark.mesos.mesosExecutor.cores ( 
https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md ) and 
spark.task.cpus ( 
https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md ) 





On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit < sumitkcha...@gmail.com > 
wrote: 



Ah thanks. looks like i skipped reading this " Neither will executors terminate 
when they’re idle." 


So in my job scenario, I should preassume that No of executors should be less 
than number of tasks. Ideally one executor should execute 1 or more tasks. But 
i am observing something strange instead. I start my job with 48 partitions for 
a spark job. In mesos ui i see that number of tasks is 48, but no. of CPUs is 
78 which is way more than 48. Here i am assuming that 1 CPU is 1 executor. I am 
not specifying any configuration to set number of cores per executor. 



Regards 
Sumit Chawla 





On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere < jo...@mesosphere.io > 
wrote: 



That makes sense. From the documentation it looks like the executors are not 
supposed to terminate: 
http://spark.apache.org/docs/latest/running-on-mesos.html#fine-grained-deprecated
 


Note that while Spark tasks in fine-grained will relinquish cores as they 
terminate, they will not relinquish memory, as the JVM does not give memory 
back to the Operating System. Neither will executors terminate when they’re 
idle. 


I suppose your task to executor CPU ratio is low enough that it looks like most 
of the resources are not being reclaimed. If your tasks were using 
significantly more CPU the amortized cost of the idle executors would not be 
such a big deal. 






— 
Joris Van Remoortere 
Mesosphere 

On Mon, Dec 19, 2016 at 11:26 AM, Timothy Chen < tnac...@gmail.com > wrote: 


Hi Chawla, 

One possible reason is that Mesos fine grain mode also takes up cores 
to run the executor per host, so if you have 20 agents running Fine 
grained executor it will take up 20 cores while it's still running. 

Tim 

On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit < sumitkcha...@gmail.com > wrote: 


> Hi 
> 
> I am using Spark 1.6. I have one query about Fine Grained model in Spark. 
> I have a simple Spark application which transforms A -> B. Its a single 
> stage application. To begin the program, It starts with 48 partitions. 
> When the program starts running, in mesos UI it shows 48 tasks and 48 CPUs 
> allocated to job. Now as the tasks get done, the number of active tasks 
> number starts decreasing. How ever, the number of CPUs does not decrease 
> propotionally. When the job was about to finish, there was a single 
> remaininig task, however CPU count was still 20. 
> 
> My questions, is why there is no one to one mapping between tasks and cpus 
> in Fine grained? How can these CPUs be released whe

Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Michael Gummelt
> Is this problem of idle executors sticking around solved in Dynamic
Resource Allocation?  Is there some timeout after which Idle executors can
just shutdown and cleanup its resources.

Yes, that's exactly what dynamic allocation does.  But again I have no idea
what the state of dynamic allocation + mesos is.

On Mon, Dec 19, 2016 at 1:32 PM, Chawla,Sumit 
wrote:

> Great.  Makes much better sense now.  What will be reason to have
> spark.mesos.mesosExecutor.cores more than 1, as this number doesn't
> include the number of cores for tasks.
>
> So in my case it seems like 30 CPUs are allocated to executors.  And there
> are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of 30 is
> maintained till the last task exits.  This explains the gap.   Thanks
> everyone.  I am still not sure how this number 30 is calculated.  ( Is it
> dynamic based on current resources, or is it some configuration.  I have 32
> nodes in my cluster).
>
> Is this problem of idle executors sticking around solved in Dynamic
> Resource Allocation?  Is there some timeout after which Idle executors can
> just shutdown and cleanup its resources.
>
>
> Regards
> Sumit Chawla
>
>
> On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt 
> wrote:
>
>> >  I should preassume that No of executors should be less than number of
>> tasks.
>>
>> No.  Each executor runs 0 or more tasks.
>>
>> Each executor consumes 1 CPU, and each task running on that executor
>> consumes another CPU.  You can customize this via
>> spark.mesos.mesosExecutor.cores (https://github.com/apache/spa
>> rk/blob/v1.6.3/docs/running-on-mesos.md) and spark.task.cpus (
>> https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md)
>>
>> On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit 
>> wrote:
>>
>>> Ah thanks. looks like i skipped reading this *"Neither will executors
>>> terminate when they’re idle."*
>>>
>>> So in my job scenario,  I should preassume that No of executors should
>>> be less than number of tasks. Ideally one executor should execute 1 or more
>>> tasks.  But i am observing something strange instead.  I start my job with
>>> 48 partitions for a spark job. In mesos ui i see that number of tasks is
>>> 48, but no. of CPUs is 78 which is way more than 48.  Here i am assuming
>>> that 1 CPU is 1 executor.   I am not specifying any configuration to set
>>> number of cores per executor.
>>>
>>> Regards
>>> Sumit Chawla
>>>
>>>
>>> On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere <
>>> jo...@mesosphere.io> wrote:
>>>
 That makes sense. From the documentation it looks like the executors
 are not supposed to terminate:
 http://spark.apache.org/docs/latest/running-on-mesos.html#fi
 ne-grained-deprecated

> Note that while Spark tasks in fine-grained will relinquish cores as
> they terminate, they will not relinquish memory, as the JVM does not give
> memory back to the Operating System. Neither will executors terminate when
> they’re idle.


 I suppose your task to executor CPU ratio is low enough that it looks
 like most of the resources are not being reclaimed. If your tasks were
 using significantly more CPU the amortized cost of the idle executors would
 not be such a big deal.


 —
 *Joris Van Remoortere*
 Mesosphere

 On Mon, Dec 19, 2016 at 11:26 AM, Timothy Chen 
 wrote:

> Hi Chawla,
>
> One possible reason is that Mesos fine grain mode also takes up cores
> to run the executor per host, so if you have 20 agents running Fine
> grained executor it will take up 20 cores while it's still running.
>
> Tim
>
> On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
> wrote:
> > Hi
> >
> > I am using Spark 1.6. I have one query about Fine Grained model in
> Spark.
> > I have a simple Spark application which transforms A -> B.  Its a
> single
> > stage application.  To begin the program, It starts with 48
> partitions.
> > When the program starts running, in mesos UI it shows 48 tasks and
> 48 CPUs
> > allocated to job.  Now as the tasks get done, the number of active
> tasks
> > number starts decreasing.  How ever, the number of CPUs does not
> decrease
> > propotionally.  When the job was about to finish, there was a single
> > remaininig task, however CPU count was still 20.
> >
> > My questions, is why there is no one to one mapping between tasks
> and cpus
> > in Fine grained?  How can these CPUs be released when the job is
> done, so
> > that other jobs can start.
> >
> >
> > Regards
> > Sumit Chawla
>


>>>
>>
>>
>> --
>> Michael Gummelt
>> Software Engineer
>> Mesosphere
>>
>
>


-- 
Michael Gummelt
Software Engineer
Mesosphere


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Chawla,Sumit
Great.  Makes much better sense now.  What will be reason to have
spark.mesos.mesosExecutor.cores more than 1, as this number doesn't include
the number of cores for tasks.

So in my case it seems like 30 CPUs are allocated to executors.  And there
are 48 tasks so 48 + 30 =  78 CPUs.  And i am noticing this gap of 30 is
maintained till the last task exits.  This explains the gap.   Thanks
everyone.  I am still not sure how this number 30 is calculated.  ( Is it
dynamic based on current resources, or is it some configuration.  I have 32
nodes in my cluster).

Is this problem of idle executors sticking around solved in Dynamic
Resource Allocation?  Is there some timeout after which Idle executors can
just shutdown and cleanup its resources.


Regards
Sumit Chawla


On Mon, Dec 19, 2016 at 12:45 PM, Michael Gummelt 
wrote:

> >  I should preassume that No of executors should be less than number of
> tasks.
>
> No.  Each executor runs 0 or more tasks.
>
> Each executor consumes 1 CPU, and each task running on that executor
> consumes another CPU.  You can customize this via 
> spark.mesos.mesosExecutor.cores
> (https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md)
> and spark.task.cpus (https://github.com/apache/spark/blob/v1.6.3/docs/
> configuration.md)
>
> On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit 
> wrote:
>
>> Ah thanks. looks like i skipped reading this *"Neither will executors
>> terminate when they’re idle."*
>>
>> So in my job scenario,  I should preassume that No of executors should be
>> less than number of tasks. Ideally one executor should execute 1 or more
>> tasks.  But i am observing something strange instead.  I start my job with
>> 48 partitions for a spark job. In mesos ui i see that number of tasks is
>> 48, but no. of CPUs is 78 which is way more than 48.  Here i am assuming
>> that 1 CPU is 1 executor.   I am not specifying any configuration to set
>> number of cores per executor.
>>
>> Regards
>> Sumit Chawla
>>
>>
>> On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere <
>> jo...@mesosphere.io> wrote:
>>
>>> That makes sense. From the documentation it looks like the executors are
>>> not supposed to terminate:
>>> http://spark.apache.org/docs/latest/running-on-mesos.html#fi
>>> ne-grained-deprecated
>>>
 Note that while Spark tasks in fine-grained will relinquish cores as
 they terminate, they will not relinquish memory, as the JVM does not give
 memory back to the Operating System. Neither will executors terminate when
 they’re idle.
>>>
>>>
>>> I suppose your task to executor CPU ratio is low enough that it looks
>>> like most of the resources are not being reclaimed. If your tasks were
>>> using significantly more CPU the amortized cost of the idle executors would
>>> not be such a big deal.
>>>
>>>
>>> —
>>> *Joris Van Remoortere*
>>> Mesosphere
>>>
>>> On Mon, Dec 19, 2016 at 11:26 AM, Timothy Chen 
>>> wrote:
>>>
 Hi Chawla,

 One possible reason is that Mesos fine grain mode also takes up cores
 to run the executor per host, so if you have 20 agents running Fine
 grained executor it will take up 20 cores while it's still running.

 Tim

 On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
 wrote:
 > Hi
 >
 > I am using Spark 1.6. I have one query about Fine Grained model in
 Spark.
 > I have a simple Spark application which transforms A -> B.  Its a
 single
 > stage application.  To begin the program, It starts with 48
 partitions.
 > When the program starts running, in mesos UI it shows 48 tasks and 48
 CPUs
 > allocated to job.  Now as the tasks get done, the number of active
 tasks
 > number starts decreasing.  How ever, the number of CPUs does not
 decrease
 > propotionally.  When the job was about to finish, there was a single
 > remaininig task, however CPU count was still 20.
 >
 > My questions, is why there is no one to one mapping between tasks and
 cpus
 > in Fine grained?  How can these CPUs be released when the job is
 done, so
 > that other jobs can start.
 >
 >
 > Regards
 > Sumit Chawla

>>>
>>>
>>
>
>
> --
> Michael Gummelt
> Software Engineer
> Mesosphere
>


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Michael Gummelt
>  I should preassume that No of executors should be less than number of
tasks.

No.  Each executor runs 0 or more tasks.

Each executor consumes 1 CPU, and each task running on that executor
consumes another CPU.  You can customize this via
spark.mesos.mesosExecutor.cores (
https://github.com/apache/spark/blob/v1.6.3/docs/running-on-mesos.md) and
spark.task.cpus (
https://github.com/apache/spark/blob/v1.6.3/docs/configuration.md)

On Mon, Dec 19, 2016 at 12:09 PM, Chawla,Sumit 
wrote:

> Ah thanks. looks like i skipped reading this *"Neither will executors
> terminate when they’re idle."*
>
> So in my job scenario,  I should preassume that No of executors should be
> less than number of tasks. Ideally one executor should execute 1 or more
> tasks.  But i am observing something strange instead.  I start my job with
> 48 partitions for a spark job. In mesos ui i see that number of tasks is
> 48, but no. of CPUs is 78 which is way more than 48.  Here i am assuming
> that 1 CPU is 1 executor.   I am not specifying any configuration to set
> number of cores per executor.
>
> Regards
> Sumit Chawla
>
>
> On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere <
> jo...@mesosphere.io> wrote:
>
>> That makes sense. From the documentation it looks like the executors are
>> not supposed to terminate:
>> http://spark.apache.org/docs/latest/running-on-mesos.html#fi
>> ne-grained-deprecated
>>
>>> Note that while Spark tasks in fine-grained will relinquish cores as
>>> they terminate, they will not relinquish memory, as the JVM does not give
>>> memory back to the Operating System. Neither will executors terminate when
>>> they’re idle.
>>
>>
>> I suppose your task to executor CPU ratio is low enough that it looks
>> like most of the resources are not being reclaimed. If your tasks were
>> using significantly more CPU the amortized cost of the idle executors would
>> not be such a big deal.
>>
>>
>> —
>> *Joris Van Remoortere*
>> Mesosphere
>>
>> On Mon, Dec 19, 2016 at 11:26 AM, Timothy Chen  wrote:
>>
>>> Hi Chawla,
>>>
>>> One possible reason is that Mesos fine grain mode also takes up cores
>>> to run the executor per host, so if you have 20 agents running Fine
>>> grained executor it will take up 20 cores while it's still running.
>>>
>>> Tim
>>>
>>> On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
>>> wrote:
>>> > Hi
>>> >
>>> > I am using Spark 1.6. I have one query about Fine Grained model in
>>> Spark.
>>> > I have a simple Spark application which transforms A -> B.  Its a
>>> single
>>> > stage application.  To begin the program, It starts with 48 partitions.
>>> > When the program starts running, in mesos UI it shows 48 tasks and 48
>>> CPUs
>>> > allocated to job.  Now as the tasks get done, the number of active
>>> tasks
>>> > number starts decreasing.  How ever, the number of CPUs does not
>>> decrease
>>> > propotionally.  When the job was about to finish, there was a single
>>> > remaininig task, however CPU count was still 20.
>>> >
>>> > My questions, is why there is no one to one mapping between tasks and
>>> cpus
>>> > in Fine grained?  How can these CPUs be released when the job is done,
>>> so
>>> > that other jobs can start.
>>> >
>>> >
>>> > Regards
>>> > Sumit Chawla
>>>
>>
>>
>


-- 
Michael Gummelt
Software Engineer
Mesosphere


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Chawla,Sumit
Ah thanks. looks like i skipped reading this *"Neither will executors
terminate when they’re idle."*

So in my job scenario,  I should preassume that No of executors should be
less than number of tasks. Ideally one executor should execute 1 or more
tasks.  But i am observing something strange instead.  I start my job with
48 partitions for a spark job. In mesos ui i see that number of tasks is
48, but no. of CPUs is 78 which is way more than 48.  Here i am assuming
that 1 CPU is 1 executor.   I am not specifying any configuration to set
number of cores per executor.

Regards
Sumit Chawla


On Mon, Dec 19, 2016 at 11:35 AM, Joris Van Remoortere 
wrote:

> That makes sense. From the documentation it looks like the executors are
> not supposed to terminate:
> http://spark.apache.org/docs/latest/running-on-mesos.html#
> fine-grained-deprecated
>
>> Note that while Spark tasks in fine-grained will relinquish cores as they
>> terminate, they will not relinquish memory, as the JVM does not give memory
>> back to the Operating System. Neither will executors terminate when they’re
>> idle.
>
>
> I suppose your task to executor CPU ratio is low enough that it looks like
> most of the resources are not being reclaimed. If your tasks were using
> significantly more CPU the amortized cost of the idle executors would not
> be such a big deal.
>
>
> —
> *Joris Van Remoortere*
> Mesosphere
>
> On Mon, Dec 19, 2016 at 11:26 AM, Timothy Chen  wrote:
>
>> Hi Chawla,
>>
>> One possible reason is that Mesos fine grain mode also takes up cores
>> to run the executor per host, so if you have 20 agents running Fine
>> grained executor it will take up 20 cores while it's still running.
>>
>> Tim
>>
>> On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
>> wrote:
>> > Hi
>> >
>> > I am using Spark 1.6. I have one query about Fine Grained model in
>> Spark.
>> > I have a simple Spark application which transforms A -> B.  Its a single
>> > stage application.  To begin the program, It starts with 48 partitions.
>> > When the program starts running, in mesos UI it shows 48 tasks and 48
>> CPUs
>> > allocated to job.  Now as the tasks get done, the number of active tasks
>> > number starts decreasing.  How ever, the number of CPUs does not
>> decrease
>> > propotionally.  When the job was about to finish, there was a single
>> > remaininig task, however CPU count was still 20.
>> >
>> > My questions, is why there is no one to one mapping between tasks and
>> cpus
>> > in Fine grained?  How can these CPUs be released when the job is done,
>> so
>> > that other jobs can start.
>> >
>> >
>> > Regards
>> > Sumit Chawla
>>
>
>


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Timothy Chen
Hi Chawla,

One possible reason is that Mesos fine grain mode also takes up cores
to run the executor per host, so if you have 20 agents running Fine
grained executor it will take up 20 cores while it's still running.

Tim

On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit  wrote:
> Hi
>
> I am using Spark 1.6. I have one query about Fine Grained model in Spark.
> I have a simple Spark application which transforms A -> B.  Its a single
> stage application.  To begin the program, It starts with 48 partitions.
> When the program starts running, in mesos UI it shows 48 tasks and 48 CPUs
> allocated to job.  Now as the tasks get done, the number of active tasks
> number starts decreasing.  How ever, the number of CPUs does not decrease
> propotionally.  When the job was about to finish, there was a single
> remaininig task, however CPU count was still 20.
>
> My questions, is why there is no one to one mapping between tasks and cpus
> in Fine grained?  How can these CPUs be released when the job is done, so
> that other jobs can start.
>
>
> Regards
> Sumit Chawla

-
To unsubscribe e-mail: user-unsubscr...@spark.apache.org



Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Michael Gummelt
Yea, the idea is to use dynamic allocation.  I can't speak to how well it
works with Mesos, though.

On Mon, Dec 19, 2016 at 11:01 AM, Mehdi Meziane <mehdi.mezi...@ldmobile.net>
wrote:

> I think that what you are looking for is Dynamic resource allocation:
> http://spark.apache.org/docs/latest/job-scheduling.html#
> dynamic-resource-allocation
>
> Spark provides a mechanism to dynamically adjust the resources your
> application occupies based on the workload. This means that your
> application may give resources back to the cluster if they are no longer
> used and request them again later when there is demand. This feature is
> particularly useful if multiple applications share resources in your Spark
> cluster.
>
> - Mail Original -
> De: "Sumit Chawla" <sumitkcha...@gmail.com>
> À: "Michael Gummelt" <mgumm...@mesosphere.io>
> Cc: u...@mesos.apache.org, "Dev" <d...@mesos.apache.org>, "User" <
> user@spark.apache.org>, "dev" <d...@spark.apache.org>
> Envoyé: Lundi 19 Décembre 2016 19h35:51 GMT +01:00 Amsterdam / Berlin /
> Berne / Rome / Stockholm / Vienne
> Objet: Re: Mesos Spark Fine Grained Execution - CPU count
>
>
> But coarse grained does the exact same thing which i am trying to avert
> here.  At the cost of lower startup, it keeps the resources reserved till
> the entire duration of the job.
>
> Regards
> Sumit Chawla
>
>
> On Mon, Dec 19, 2016 at 10:06 AM, Michael Gummelt <mgumm...@mesosphere.io>
> wrote:
>
>> Hi
>>
>> I don't have a lot of experience with the fine-grained scheduler.  It's
>> deprecated and fairly old now.  CPUs should be relinquished as tasks
>> complete, so I'm not sure why you're seeing what you're seeing.  There have
>> been a few discussions on the spark list regarding deprecating the
>> fine-grained scheduler, and no one seemed too dead-set on keeping it.  I'd
>> recommend you move over to coarse-grained.
>>
>> On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit <sumitkcha...@gmail.com>
>> wrote:
>>
>>> Hi
>>>
>>> I am using Spark 1.6. I have one query about Fine Grained model in
>>> Spark.  I have a simple Spark application which transforms A -> B.  Its a
>>> single stage application.  To begin the program, It starts with 48
>>> partitions.  When the program starts running, in mesos UI it shows 48 tasks
>>> and 48 CPUs allocated to job.  Now as the tasks get done, the number of
>>> active tasks number starts decreasing.  How ever, the number of CPUs does
>>> not decrease propotionally.  When the job was about to finish, there was a
>>> single remaininig task, however CPU count was still 20.
>>>
>>> My questions, is why there is no one to one mapping between tasks and
>>> cpus in Fine grained?  How can these CPUs be released when the job is done,
>>> so that other jobs can start.
>>>
>>>
>>> Regards
>>> Sumit Chawla
>>>
>>>
>>
>>
>> --
>> Michael Gummelt
>> Software Engineer
>> Mesosphere
>>
>
>


-- 
Michael Gummelt
Software Engineer
Mesosphere


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Mehdi Meziane
I think that what you are looking for is Dynamic resource allocation: 
http://spark.apache.org/docs/latest/job-scheduling.html#dynamic-resource-allocation
 


Spark provides a mechanism to dynamically adjust the resources your application 
occupies based on the workload. This means that your application may give 
resources back to the cluster if they are no longer used and request them again 
later when there is demand. This feature is particularly useful if multiple 
applications share resources in your Spark cluster. 

- Mail Original - 
De: "Sumit Chawla" <sumitkcha...@gmail.com> 
À: "Michael Gummelt" <mgumm...@mesosphere.io> 
Cc: u...@mesos.apache.org, "Dev" <d...@mesos.apache.org>, "User" 
<user@spark.apache.org>, "dev" <d...@spark.apache.org> 
Envoyé: Lundi 19 Décembre 2016 19h35:51 GMT +01:00 Amsterdam / Berlin / Berne / 
Rome / Stockholm / Vienne 
Objet: Re: Mesos Spark Fine Grained Execution - CPU count 


But coarse grained does the exact same thing which i am trying to avert here. 
At the cost of lower startup, it keeps the resources reserved till the entire 
duration of the job. 



Regards 
Sumit Chawla 



On Mon, Dec 19, 2016 at 10:06 AM, Michael Gummelt < mgumm...@mesosphere.io > 
wrote: 




Hi 

I don't have a lot of experience with the fine-grained scheduler. It's 
deprecated and fairly old now. CPUs should be relinquished as tasks complete, 
so I'm not sure why you're seeing what you're seeing. There have been a few 
discussions on the spark list regarding deprecating the fine-grained scheduler, 
and no one seemed too dead-set on keeping it. I'd recommend you move over to 
coarse-grained. 





On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit < sumitkcha...@gmail.com > wrote: 



Hi 


I am using Spark 1.6. I have one query about Fine Grained model in Spark. I 
have a simple Spark application which transforms A -> B. Its a single stage 
application. To begin the program, It starts with 48 partitions. When the 
program starts running, in mesos UI it shows 48 tasks and 48 CPUs allocated to 
job. Now as the tasks get done, the number of active tasks number starts 
decreasing. How ever, the number of CPUs does not decrease propotionally. When 
the job was about to finish, there was a single remaininig task, however CPU 
count was still 20. 


My questions, is why there is no one to one mapping between tasks and cpus in 
Fine grained? How can these CPUs be released when the job is done, so that 
other jobs can start. 






Regards 
Sumit Chawla 




-- 







Michael Gummelt 
Software Engineer 
Mesosphere 



Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Chawla,Sumit
But coarse grained does the exact same thing which i am trying to avert
here.  At the cost of lower startup, it keeps the resources reserved till
the entire duration of the job.

Regards
Sumit Chawla


On Mon, Dec 19, 2016 at 10:06 AM, Michael Gummelt 
wrote:

> Hi
>
> I don't have a lot of experience with the fine-grained scheduler.  It's
> deprecated and fairly old now.  CPUs should be relinquished as tasks
> complete, so I'm not sure why you're seeing what you're seeing.  There have
> been a few discussions on the spark list regarding deprecating the
> fine-grained scheduler, and no one seemed too dead-set on keeping it.  I'd
> recommend you move over to coarse-grained.
>
> On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
> wrote:
>
>> Hi
>>
>> I am using Spark 1.6. I have one query about Fine Grained model in
>> Spark.  I have a simple Spark application which transforms A -> B.  Its a
>> single stage application.  To begin the program, It starts with 48
>> partitions.  When the program starts running, in mesos UI it shows 48 tasks
>> and 48 CPUs allocated to job.  Now as the tasks get done, the number of
>> active tasks number starts decreasing.  How ever, the number of CPUs does
>> not decrease propotionally.  When the job was about to finish, there was a
>> single remaininig task, however CPU count was still 20.
>>
>> My questions, is why there is no one to one mapping between tasks and
>> cpus in Fine grained?  How can these CPUs be released when the job is done,
>> so that other jobs can start.
>>
>>
>> Regards
>> Sumit Chawla
>>
>>
>
>
> --
> Michael Gummelt
> Software Engineer
> Mesosphere
>


Re: Mesos Spark Fine Grained Execution - CPU count

2016-12-19 Thread Michael Gummelt
Hi

I don't have a lot of experience with the fine-grained scheduler.  It's
deprecated and fairly old now.  CPUs should be relinquished as tasks
complete, so I'm not sure why you're seeing what you're seeing.  There have
been a few discussions on the spark list regarding deprecating the
fine-grained scheduler, and no one seemed too dead-set on keeping it.  I'd
recommend you move over to coarse-grained.

On Fri, Dec 16, 2016 at 8:41 AM, Chawla,Sumit 
wrote:

> Hi
>
> I am using Spark 1.6. I have one query about Fine Grained model in Spark.
> I have a simple Spark application which transforms A -> B.  Its a single
> stage application.  To begin the program, It starts with 48 partitions.
> When the program starts running, in mesos UI it shows 48 tasks and 48 CPUs
> allocated to job.  Now as the tasks get done, the number of active tasks
> number starts decreasing.  How ever, the number of CPUs does not decrease
> propotionally.  When the job was about to finish, there was a single
> remaininig task, however CPU count was still 20.
>
> My questions, is why there is no one to one mapping between tasks and cpus
> in Fine grained?  How can these CPUs be released when the job is done, so
> that other jobs can start.
>
>
> Regards
> Sumit Chawla
>
>


-- 
Michael Gummelt
Software Engineer
Mesosphere


Re: Mesos + Spark

2015-07-24 Thread boci
Thanks, but something is not clear...
I have the mesos cluster.
- I want to submit my application and scheduled with chronos.
- For cluster mode I need a dispatcher, this is another container (machine
in the real world)? What will this do? It's needed when I using chronos?
- How can I access to my spark job from chronos?

I think submit in client mode is not fit to my condition, that's right?

Thanks
b0c1

--
Skype: boci13, Hangout: boci.b...@gmail.com

On Wed, Jul 22, 2015 at 4:51 PM, Dean Wampler deanwamp...@gmail.com wrote:

 This page, http://spark.apache.org/docs/latest/running-on-mesos.html,
 covers many of these questions. If you submit a job with the option
 --supervise, it will be restarted if it fails.

 You can use Chronos for scheduling. You can create a single streaming job
 with a 10 minute batch interval, if that works for your every 10-min. need.

 Dean Wampler, Ph.D.
 Author: Programming Scala, 2nd Edition
 http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
 Typesafe http://typesafe.com
 @deanwampler http://twitter.com/deanwampler
 http://polyglotprogramming.com

 On Wed, Jul 22, 2015 at 3:53 AM, boci boci.b...@gmail.com wrote:

 Hi guys!

 I'm a new in mesos. I have two spark application (one streaming and one
 batch). I want to run both app in mesos cluster. Now for testing I want to
 run in docker container so I started a simple redjack/mesos-master, but I
 think a lot of think unclear for me (both mesos and spark-mesos).

 If I have a mesos cluster (for testing it will be some docker container)
 i need a separate machine (container) to run my spark job? Or can I submit
 the cluster and schedule (chronos or I dunno)?
 How can I run the streaming job? What happened if the controller died?
 Or if I call spark-submit with master=mesos my application started and I
 can forget? How can I run in every 10 min without submit in every 10 min?
 How can I run my streaming app in HA mode?

 Thanks

 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com





Re: Mesos + Spark

2015-07-24 Thread Dean Wampler
When running Spark in Mesos cluster mode, the driver program runs in one of
the cluster nodes, like the other Spark processes that are spawned. You
won't need a special node for this purpose. I'm not very familiar with
Chronos, but its UI or the regular Mesos UI should show you where the
driver is running, then you can use the Spark web UI on that machine to see
what the Spark job is doing.

dean

Dean Wampler, Ph.D.
Author: Programming Scala, 2nd Edition
http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
Typesafe http://typesafe.com
@deanwampler http://twitter.com/deanwampler
http://polyglotprogramming.com

On Fri, Jul 24, 2015 at 4:47 PM, boci boci.b...@gmail.com wrote:

 Thanks, but something is not clear...
 I have the mesos cluster.
 - I want to submit my application and scheduled with chronos.
 - For cluster mode I need a dispatcher, this is another container (machine
 in the real world)? What will this do? It's needed when I using chronos?
 - How can I access to my spark job from chronos?

 I think submit in client mode is not fit to my condition, that's right?

 Thanks
 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com

 On Wed, Jul 22, 2015 at 4:51 PM, Dean Wampler deanwamp...@gmail.com
 wrote:

 This page, http://spark.apache.org/docs/latest/running-on-mesos.html,
 covers many of these questions. If you submit a job with the option
 --supervise, it will be restarted if it fails.

 You can use Chronos for scheduling. You can create a single streaming job
 with a 10 minute batch interval, if that works for your every 10-min. need.

 Dean Wampler, Ph.D.
 Author: Programming Scala, 2nd Edition
 http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
 Typesafe http://typesafe.com
 @deanwampler http://twitter.com/deanwampler
 http://polyglotprogramming.com

 On Wed, Jul 22, 2015 at 3:53 AM, boci boci.b...@gmail.com wrote:

 Hi guys!

 I'm a new in mesos. I have two spark application (one streaming and one
 batch). I want to run both app in mesos cluster. Now for testing I want to
 run in docker container so I started a simple redjack/mesos-master, but I
 think a lot of think unclear for me (both mesos and spark-mesos).

 If I have a mesos cluster (for testing it will be some docker container)
 i need a separate machine (container) to run my spark job? Or can I submit
 the cluster and schedule (chronos or I dunno)?
 How can I run the streaming job? What happened if the controller died?
 Or if I call spark-submit with master=mesos my application started and I
 can forget? How can I run in every 10 min without submit in every 10 min?
 How can I run my streaming app in HA mode?

 Thanks

 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com






Re: Mesos + Spark

2015-07-24 Thread Dean Wampler
You can certainly start jobs without Chronos, but to automatically restart
finished jobs or to run jobs at specific times or periods, you'll want
something like Chronos.

dean

Dean Wampler, Ph.D.
Author: Programming Scala, 2nd Edition
http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
Typesafe http://typesafe.com
@deanwampler http://twitter.com/deanwampler
http://polyglotprogramming.com

On Fri, Jul 24, 2015 at 5:08 PM, boci boci.b...@gmail.com wrote:

 Thanks,
 Mesos will show spark is driver is running, but what happened if my batch
 job finished? How can I reschedule without chronos ? Can I submit a job
 without start it?

 Thanks

 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com

 On Fri, Jul 24, 2015 at 11:52 PM, Dean Wampler deanwamp...@gmail.com
 wrote:

 When running Spark in Mesos cluster mode, the driver program runs in one
 of the cluster nodes, like the other Spark processes that are spawned. You
 won't need a special node for this purpose. I'm not very familiar with
 Chronos, but its UI or the regular Mesos UI should show you where the
 driver is running, then you can use the Spark web UI on that machine to see
 what the Spark job is doing.

 dean

 Dean Wampler, Ph.D.
 Author: Programming Scala, 2nd Edition
 http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
 Typesafe http://typesafe.com
 @deanwampler http://twitter.com/deanwampler
 http://polyglotprogramming.com

 On Fri, Jul 24, 2015 at 4:47 PM, boci boci.b...@gmail.com wrote:

 Thanks, but something is not clear...
 I have the mesos cluster.
 - I want to submit my application and scheduled with chronos.
 - For cluster mode I need a dispatcher, this is another container
 (machine in the real world)? What will this do? It's needed when I using
 chronos?
 - How can I access to my spark job from chronos?

 I think submit in client mode is not fit to my condition, that's right?

 Thanks
 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com

 On Wed, Jul 22, 2015 at 4:51 PM, Dean Wampler deanwamp...@gmail.com
 wrote:

 This page, http://spark.apache.org/docs/latest/running-on-mesos.html,
 covers many of these questions. If you submit a job with the option
 --supervise, it will be restarted if it fails.

 You can use Chronos for scheduling. You can create a single streaming
 job with a 10 minute batch interval, if that works for your every 10-min.
 need.

 Dean Wampler, Ph.D.
 Author: Programming Scala, 2nd Edition
 http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
 Typesafe http://typesafe.com
 @deanwampler http://twitter.com/deanwampler
 http://polyglotprogramming.com

 On Wed, Jul 22, 2015 at 3:53 AM, boci boci.b...@gmail.com wrote:

 Hi guys!

 I'm a new in mesos. I have two spark application (one streaming and
 one batch). I want to run both app in mesos cluster. Now for testing I 
 want
 to run in docker container so I started a simple redjack/mesos-master, but
 I think a lot of think unclear for me (both mesos and spark-mesos).

 If I have a mesos cluster (for testing it will be some docker
 container) i need a separate machine (container) to run my spark job? Or
 can I submit the cluster and schedule (chronos or I dunno)?
 How can I run the streaming job? What happened if the controller
 died? Or if I call spark-submit with master=mesos my application started
 and I can forget? How can I run in every 10 min without submit in every 10
 min? How can I run my streaming app in HA mode?

 Thanks

 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com








Re: Mesos + Spark

2015-07-22 Thread Dean Wampler
This page, http://spark.apache.org/docs/latest/running-on-mesos.html,
covers many of these questions. If you submit a job with the option
--supervise, it will be restarted if it fails.

You can use Chronos for scheduling. You can create a single streaming job
with a 10 minute batch interval, if that works for your every 10-min. need.

Dean Wampler, Ph.D.
Author: Programming Scala, 2nd Edition
http://shop.oreilly.com/product/0636920033073.do (O'Reilly)
Typesafe http://typesafe.com
@deanwampler http://twitter.com/deanwampler
http://polyglotprogramming.com

On Wed, Jul 22, 2015 at 3:53 AM, boci boci.b...@gmail.com wrote:

 Hi guys!

 I'm a new in mesos. I have two spark application (one streaming and one
 batch). I want to run both app in mesos cluster. Now for testing I want to
 run in docker container so I started a simple redjack/mesos-master, but I
 think a lot of think unclear for me (both mesos and spark-mesos).

 If I have a mesos cluster (for testing it will be some docker container) i
 need a separate machine (container) to run my spark job? Or can I submit
 the cluster and schedule (chronos or I dunno)?
 How can I run the streaming job? What happened if the controller died?
 Or if I call spark-submit with master=mesos my application started and I
 can forget? How can I run in every 10 min without submit in every 10 min?
 How can I run my streaming app in HA mode?

 Thanks

 b0c1


 --
 Skype: boci13, Hangout: boci.b...@gmail.com



Re: Mesos Spark Tasks - Lost

2015-05-20 Thread Tim Chen
Can you share your exact spark-submit command line?

And also cluster mode is not yet released yet (1.4) and doesn't support
spark-shell, so I think you're just using client mode unless you're using
latest master.

Tim

On Tue, May 19, 2015 at 8:57 AM, Panagiotis Garefalakis panga...@gmail.com
wrote:

 Hello all,

 I am facing a weird issue for the last couple of days running Spark on top
 of Mesos and I need your help. I am running Mesos in a private cluster and
 managed to deploy successfully  hdfs, cassandra, marathon and play but
 Spark is not working for a reason. I have tried so far:
 different java versions (1.6 and 1.7 oracle and openjdk), different
 spark-env configuration, different Spark versions (from 0.8.8 to 1.3.1),
 different HDFS versions (hadoop 5.1 and 4.6), and updating pom dependencies.

 More specifically while local tasks complete fine, in cluster mode all the
 tasks get lost.
 (both using spark-shell and spark-submit)
 From the worker log I see something like this:

 ---
 I0519 02:36:30.475064 12863 fetcher.cpp:214] Fetching URI
 'hdfs:/:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 I0519 02:36:30.747372 12863 fetcher.cpp:99] Fetching URI
 'hdfs://X:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz' using Hadoop
 Client
 I0519 02:36:30.747546 12863 fetcher.cpp:109] Downloading resource from
 'hdfs://:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz' to
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 I0519 02:36:34.205878 12863 fetcher.cpp:78] Extracted resource
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 into
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3'
 *Error: Could not find or load main class two*

 ---

 And from the Spark Terminal:

 ---
 15/05/19 02:36:39 INFO scheduler.TaskSchedulerImpl: Cancelling stage 0
 15/05/19 02:36:39 INFO scheduler.TaskSchedulerImpl: Stage 0 was cancelled
 15/05/19 02:36:39 INFO scheduler.DAGScheduler: Failed to run reduce at
 SparkPi.scala:35
 15/05/19 02:36:39 INFO scheduler.DAGScheduler: Failed to run reduce at
 SparkPi.scala:35
 Exception in thread main org.apache.spark.SparkException: Job aborted
 due to stage failure: Task 7 in stage 0.0 failed 4 times, most recent
 failure: Lost task 7.3 in stage 0.0 (TID 26, ): ExecutorLostFailure
 (executor lost)
 Driver stacktrace: at
 org.apache.spark.scheduler.DAGScheduler.org
 http://org.apache.spark.scheduler.dagscheduler.org/$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1185)at
 org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1174)atorg.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1173)at
 scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
 ..
 at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
 at
 scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 ---

 Any help will be greatly appreciated!

 Regards,
 Panagiotis



Re: Mesos Spark Tasks - Lost

2015-05-20 Thread Panagiotis Garefalakis
Tim thanks for your reply,

I am following this quite clear mesos-spark tutorial:
https://docs.mesosphere.com/tutorials/run-spark-on-mesos/
So mainly I tried running spark-shell which locally works fine but when the
jobs are submitted through mesos something goes wrong!

My question is: is there a some extra configuration needed for the workers
(that is not mentioned at the tutorial) ??

The Executor Lost message I get is really generic so I dont know whats
going on..
Please check the attached mesos execution event log.

Thanks again,
Panagiotis


On Wed, May 20, 2015 at 8:21 AM, Tim Chen t...@mesosphere.io wrote:

 Can you share your exact spark-submit command line?

 And also cluster mode is not yet released yet (1.4) and doesn't support
 spark-shell, so I think you're just using client mode unless you're using
 latest master.

 Tim

 On Tue, May 19, 2015 at 8:57 AM, Panagiotis Garefalakis 
 panga...@gmail.com wrote:

 Hello all,

 I am facing a weird issue for the last couple of days running Spark on
 top of Mesos and I need your help. I am running Mesos in a private cluster
 and managed to deploy successfully  hdfs, cassandra, marathon and play but
 Spark is not working for a reason. I have tried so far:
 different java versions (1.6 and 1.7 oracle and openjdk), different
 spark-env configuration, different Spark versions (from 0.8.8 to 1.3.1),
 different HDFS versions (hadoop 5.1 and 4.6), and updating pom dependencies.

 More specifically while local tasks complete fine, in cluster mode all
 the tasks get lost.
 (both using spark-shell and spark-submit)
 From the worker log I see something like this:

 ---
 I0519 02:36:30.475064 12863 fetcher.cpp:214] Fetching URI
 'hdfs:/:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 I0519 02:36:30.747372 12863 fetcher.cpp:99] Fetching URI
 'hdfs://X:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz' using Hadoop
 Client
 I0519 02:36:30.747546 12863 fetcher.cpp:109] Downloading resource from
 'hdfs://:8020/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz' to
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 I0519 02:36:34.205878 12863 fetcher.cpp:78] Extracted resource
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3/spark-1.1.0-bin-2.0.0-cdh4.7.0.tgz'
 into
 '/tmp/mesos/slaves/20150515-164602-2877535122-5050-32131-S2/frameworks/20150517-162701-2877535122-5050-28705-0084/executors/20150515-164602-2877535122-5050-32131-S2/runs/660d78ec-e2f4-4d38-881b-7209cbd3c5c3'
 *Error: Could not find or load main class two*

 ---

 And from the Spark Terminal:

 ---
 15/05/19 02:36:39 INFO scheduler.TaskSchedulerImpl: Cancelling stage 0
 15/05/19 02:36:39 INFO scheduler.TaskSchedulerImpl: Stage 0 was cancelled
 15/05/19 02:36:39 INFO scheduler.DAGScheduler: Failed to run reduce at
 SparkPi.scala:35
 15/05/19 02:36:39 INFO scheduler.DAGScheduler: Failed to run reduce at
 SparkPi.scala:35
 Exception in thread main org.apache.spark.SparkException: Job aborted
 due to stage failure: Task 7 in stage 0.0 failed 4 times, most recent
 failure: Lost task 7.3 in stage 0.0 (TID 26, ): ExecutorLostFailure
 (executor lost)
 Driver stacktrace: at
 org.apache.spark.scheduler.DAGScheduler.org
 http://org.apache.spark.scheduler.dagscheduler.org/$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1185)at
 org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1174)atorg.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1173)at
 scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
 ..
 at
 scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
 at
 scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 ---

 Any help will be greatly appreciated!

 Regards,
 Panagiotis





-sparklogs-spark-shell-1431993674182-EVENT_LOG_1
Description: Binary data

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org