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 ! >> >> >> > >> >> >> > >> >> >> > ----- Mail Original ----- >> >> >> > De: "Michael Gummelt" <mgumm...@mesosphere.io> >> >> >> > À: "Sumit Chawla" <sumitkcha...@gmail.com> >> >> >> > Cc: u...@mesos.apache.org, d...@mesos.apache.org, "User" >> >> >> > <u...@spark.apache.org>, 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 >> >> >> > <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 when the >> >> >> >>>>>> > job >> >> >> >>>>>> > is >> >> >> >>>>>> > done, so >> >> >> >>>>>> > that other jobs can start. >> >> >> >>>>>> > >> >> >> >>>>>> > >> >> >> >>>>>> > Regards >> >> >> >>>>>> > Sumit Chawla >> >> >> >>>>> >> >> >> >>>>> >> >> >> >>>> >> >> >> >>> >> >> >> >>> >> >> >> >>> >> >> >> >>> -- >> >> >> >>> Michael Gummelt >> >> >> >>> Software Engineer >> >> >> >>> Mesosphere >> >> >> >> >> >> >> >> >> >> >> > >> >> >> > >> >> >> > >> >> >> > -- >> >> >> > Michael Gummelt >> >> >> > Software Engineer >> >> >> > Mesosphere >> >> >> >> >> >> >> >> >> >> >> -- >> >> - Davies >> > >> > >> > >> > >> > -- >> > Michael Gummelt >> > Software Engineer >> > Mesosphere > > > > > -- > Michael Gummelt > Software Engineer > Mesosphere --------------------------------------------------------------------- To unsubscribe e-mail: dev-unsubscr...@spark.apache.org