Yeah, agreed. Only for some extreme streaming workload we designed to fit
the pattern of dynamic allocation that could be worked very well. In normal
cases, no executor will remain idle for long time, so frequently scale up
and ramp down of executors will bring large overhead and latency to
streaming app.

On Thu, Nov 12, 2015 at 10:43 AM, Tathagata Das <t...@databricks.com> wrote:

> For receivers, you must enable write ahead logs for making sure you dont
> loose data.
> And tuning the backlog time, executor idle time etc would not work very
> well for streaming, as the micro-batch jobs are likely to use all the
> executors all the time, and no executor will remain idle for long. That is
> why the heuristic doesnt work that well.
>
>
> On Wed, Nov 11, 2015 at 6:32 PM, Saisai Shao <sai.sai.s...@gmail.com>
> wrote:
>
>> I think for receiver-less Streaming connectors like direct Kafka input
>> stream or hdfs connector, dynamic allocation could be worked compared to
>> other receiver-based streaming connectors, since for receiver-less
>> connectors, the behavior of streaming app is more like a normal Spark app,
>> so dynamic allocation could be worked well, but you have to tune the
>> backlog time, executor idle time as well as batch duration to make dynamic
>> allocation work well in streaming app.
>>
>> Thanks
>> Jerry
>>
>>
>> On Thu, Nov 12, 2015 at 5:36 AM, PhuDuc Nguyen <duc.was.h...@gmail.com>
>> wrote:
>>
>>> Awesome, thanks for the tip!
>>>
>>>
>>>
>>> On Wed, Nov 11, 2015 at 2:25 PM, Tathagata Das <t...@databricks.com>
>>> wrote:
>>>
>>>> The reason the existing dynamic allocation does not work out of the box
>>>> for spark streaming is because the heuristics used for decided when to
>>>> scale up/down is not the right one for micro-batch workloads. It works
>>>> great for typical batch workloads. However you can use the underlying
>>>> developer API to add / remove executors to implement your own scaling
>>>> logic.
>>>>
>>>> 1. Use SparkContext.requestExecutor and SparkContext.killExecutor
>>>>
>>>> 2. Use StreamingListener to get the scheduling delay and processing
>>>> times, and use that do a request or kill executors.
>>>>
>>>> TD
>>>>
>>>> On Wed, Nov 11, 2015 at 9:48 AM, PhuDuc Nguyen <duc.was.h...@gmail.com>
>>>> wrote:
>>>>
>>>>> Dean,
>>>>>
>>>>> Thanks for the reply. I'm searching (via spark mailing list archive
>>>>> and google) and can't find the previous thread you mentioned. I've 
>>>>> stumbled
>>>>> upon a few but may not be the thread you're referring to. I'm very
>>>>> interested in reading that discussion and any links/keywords would be
>>>>> greatly appreciated.
>>>>>
>>>>> I can see it's a non-trivial problem to solve for every use case in
>>>>> streaming and thus not yet supported in general. However, I think (maybe
>>>>> naively) it can be solved for specific use cases. If I use the available
>>>>> features to create a fault tolerant design - i.e. failures/dead nodes can
>>>>> occur on master nodes, driver node, or executor nodes without data loss 
>>>>> and
>>>>> "at-least-once" semantics is acceptable - then can't I safely scale down 
>>>>> in
>>>>> streaming by killing executors? If this is not true, then are we saying
>>>>> that streaming is not fault tolerant?
>>>>>
>>>>> I know it won't be as simple as setting a config like
>>>>> spark.dyanmicAllocation.enabled=true and magically we'll have elastic
>>>>> streaming, but I'm interested if anyone else has attempted to solve this
>>>>> for their specific use case with extra coding involved? Pitfalls? 
>>>>> Thoughts?
>>>>>
>>>>> thanks,
>>>>> Duc
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Nov 11, 2015 at 8:36 AM, Dean Wampler <deanwamp...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Dynamic allocation doesn't work yet with Spark Streaming in any
>>>>>> cluster scenario. There was a previous thread on this topic which 
>>>>>> discusses
>>>>>> the issues that need to be resolved.
>>>>>>
>>>>>> 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, Nov 11, 2015 at 8:09 AM, PhuDuc Nguyen <
>>>>>> duc.was.h...@gmail.com> wrote:
>>>>>>
>>>>>>> I'm trying to get Spark Streaming to scale up/down its number of
>>>>>>> executors within Mesos based on workload. It's not scaling down. I'm 
>>>>>>> using
>>>>>>> Spark 1.5.1 reading from Kafka using the direct (receiver-less) 
>>>>>>> approach.
>>>>>>>
>>>>>>> Based on this ticket
>>>>>>> https://issues.apache.org/jira/browse/SPARK-6287 with the right
>>>>>>> configuration, I have a simple example working with the spark-shell
>>>>>>> connected to a Mesos cluster. By working I mean the number of executors
>>>>>>> scales up/down based on workload. However, the spark-shell is not a
>>>>>>> streaming example.
>>>>>>>
>>>>>>> What is that status of dynamic resource allocation with Spark
>>>>>>> Streaming on Mesos? Is it supported at all? Or supported but with some
>>>>>>> caveats to ensure no data loss?
>>>>>>>
>>>>>>> thanks,
>>>>>>> Duc
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to