Re: Launching multiple spark jobs within a main spark job.

2016-12-24 Thread Naveen
Thanks Liang, Vadim and everyone for your inputs!!

With this clarity, I've tried client modes for both main and sub-spark
jobs. Every main spark job and its corresponding threaded spark jobs are
coming up on the YARN applications list and the jobs are getting executed
properly. I need to now test with cluster modes at both levels, and need to
setup spark-submit and few configurations properly on all data nodes in the
cluster. I will share the updates as and when I execute and analyze further.

Concern now which I am thinking is: how to throttle multiple jobs launching
based on the YARN cluster's availability. This exercise will be similar to
performing cluster's break-point analysis. But problem here is that we will
not know the file sizes until we read and get in memory and since Spark's
memory mechanics are more subtle and fragile, need to be 100% sure and
avoid OOM (out-of-memory) issues. Not sure if there is any process
available which can poll resource manager's information and tell if any
further jobs can be submitted to YARN.


On Thu, Dec 22, 2016 at 7:26 AM, Liang-Chi Hsieh  wrote:

>
> If you run the main driver and other Spark jobs in client mode, you can
> make
> sure they (I meant all the drivers) are running at the same node. Of course
> all drivers now consume the resources at the same node.
>
> If you run the main driver in client mode, but run other Spark jobs in
> cluster mode, the drivers of those Spark jobs will be launched at other
> nodes in the cluster. It should work too. It is as same as you run a Spark
> app in client mode and more others in cluster mode.
>
> If you run your main driver in cluster mode, and run other Spark jobs in
> cluster mode too, you may need  Spark properly installed in all nodes in
> the
> cluster, because those Spark jobs will be launched at the node which the
> main driver is running on.
>
>
>
>
>
> -
> Liang-Chi Hsieh | @viirya
> Spark Technology Center
> http://www.spark.tc/
> --
> View this message in context: http://apache-spark-
> developers-list.1001551.n3.nabble.com/Launching-multiple-
> spark-jobs-within-a-main-spark-job-tp20311p20327.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> -
> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>
>


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  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  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
>>  wrote:
>> > We will be interested by the results if you give a try to Dynamic
>> allocation
>> > with mesos !
>> >
>> >
>> > - Mail Original -
>> > De: "Michael Gummelt" 
>> > À: "Sumit Chawla" 
>> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User"
>> > , dev@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 
>> > 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 > >
>> >>> 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
>>   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
>> > s