Processing stuff in batch is not the same thing as being transactional. If
you look at Storm, it will e.g. skip tuples that were already applied to a
state to avoid counting stuff twice etc. Spark doesn't come with such
facility, so you could end up counting twice etc.



On Wed, Jun 17, 2015 at 2:09 PM, Ashish Soni <[email protected]> wrote:

> Stream can also be processed in micro-batch / batches which is the main
> reason behind Spark Steaming so what is the difference ?
>
> Ashish
>
> On Wed, Jun 17, 2015 at 9:04 AM, Enno Shioji <[email protected]> wrote:
>
>> PS just to elaborate on my first sentence, the reason Spark (not
>> streaming) can offer exactly once semantics is because its update operation
>> is idempotent. This is easy to do in a batch context because the input is
>> finite, but it's harder in streaming context.
>>
>> On Wed, Jun 17, 2015 at 2:00 PM, Enno Shioji <[email protected]> wrote:
>>
>>> So Spark (not streaming) does offer exactly once. Spark Streaming
>>> however, can only do exactly once semantics *if the update operation is
>>> idempotent*. updateStateByKey's update operation is idempotent, because
>>> it completely replaces the previous state.
>>>
>>> So as long as you use Spark streaming, you must somehow make the update
>>> operation idempotent. Replacing the entire state is the easiest way to do
>>> it, but it's obviously expensive.
>>>
>>> The alternative is to do something similar to what Storm does. At that
>>> point, you'll have to ask though if just using Storm is easier than that.
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Jun 17, 2015 at 1:50 PM, Ashish Soni <[email protected]>
>>> wrote:
>>>
>>>> As per my Best Understanding Spark Streaming offer Exactly once
>>>> processing , is this achieve only through updateStateByKey or there is
>>>> another way to do the same.
>>>>
>>>> Ashish
>>>>
>>>> On Wed, Jun 17, 2015 at 8:48 AM, Enno Shioji <[email protected]> wrote:
>>>>
>>>>> In that case I assume you need exactly once semantics. There's no
>>>>> out-of-the-box way to do that in Spark. There is updateStateByKey, but 
>>>>> it's
>>>>> not practical with your use case as the state is too large (it'll try to
>>>>> dump the entire intermediate state on every checkpoint, which would be
>>>>> prohibitively expensive).
>>>>>
>>>>> So either you have to implement something yourself, or you can use
>>>>> Storm Trident (or transactional low-level API).
>>>>>
>>>>> On Wed, Jun 17, 2015 at 1:26 PM, Ashish Soni <[email protected]>
>>>>> wrote:
>>>>>
>>>>>> My Use case is below
>>>>>>
>>>>>> We are going to receive lot of event as stream ( basically Kafka
>>>>>> Stream ) and then we need to process and compute
>>>>>>
>>>>>> Consider you have a phone contract with ATT and every call / sms /
>>>>>> data useage you do is an event and then it needs  to calculate your bill 
>>>>>> on
>>>>>> real time basis so when you login to your account you can see all those
>>>>>> variable as how much you used and how much is left and what is your bill
>>>>>> till date ,Also there are different rules which need to be considered 
>>>>>> when
>>>>>> you calculate the total bill one simple rule will be 0-500 min it is free
>>>>>> but above it is $1 a min.
>>>>>>
>>>>>> How do i maintain a shared state  ( total amount , total min , total
>>>>>> data etc ) so that i know how much i accumulated at any given point as
>>>>>> events for same phone can go to any node / executor.
>>>>>>
>>>>>> Can some one please tell me how can i achieve this is spark as in
>>>>>> storm i can have a bolt which can do this ?
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Jun 17, 2015 at 4:52 AM, Enno Shioji <[email protected]>
>>>>>> wrote:
>>>>>>
>>>>>>> I guess both. In terms of syntax, I was comparing it with Trident.
>>>>>>>
>>>>>>> If you are joining, Spark Streaming actually does offer windowed
>>>>>>> join out of the box. We couldn't use this though as our event stream can
>>>>>>> grow "out-of-sync", so we had to implement something on top of Storm. If
>>>>>>> your event streams don't become out of sync, you may find the built-in 
>>>>>>> join
>>>>>>> in Spark Streaming useful. Storm also has a join keyword but its 
>>>>>>> semantics
>>>>>>> are different.
>>>>>>>
>>>>>>>
>>>>>>> > Also, what do you mean by "No Back Pressure" ?
>>>>>>>
>>>>>>> So when a topology is overloaded, Storm is designed so that it will
>>>>>>> stop reading from the source. Spark on the other hand, will keep reading
>>>>>>> from the source and spilling it internally. This maybe fine, in 
>>>>>>> fairness,
>>>>>>> but it does mean you have to worry about the persistent store usage in 
>>>>>>> the
>>>>>>> processing cluster, whereas with Storm you don't have to worry because 
>>>>>>> the
>>>>>>> messages just remain in the data store.
>>>>>>>
>>>>>>> Spark came up with the idea of rate limiting, but I don't feel this
>>>>>>> is as nice as back pressure because it's very difficult to tune it such
>>>>>>> that you don't cap the cluster's processing power but yet so that it 
>>>>>>> will
>>>>>>> prevent the persistent storage to get used up.
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Jun 17, 2015 at 9:33 AM, Spark Enthusiast <
>>>>>>> [email protected]> wrote:
>>>>>>>
>>>>>>>> When you say Storm, did you mean Storm with Trident or Storm?
>>>>>>>>
>>>>>>>> My use case does not have simple transformation. There are complex
>>>>>>>> events that need to be generated by joining the incoming event stream.
>>>>>>>>
>>>>>>>> Also, what do you mean by "No Back PRessure" ?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>   On Wednesday, 17 June 2015 11:57 AM, Enno Shioji <
>>>>>>>> [email protected]> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> We've evaluated Spark Streaming vs. Storm and ended up sticking
>>>>>>>> with Storm.
>>>>>>>>
>>>>>>>> Some of the important draw backs are:
>>>>>>>> Spark has no back pressure (receiver rate limit can alleviate this
>>>>>>>> to a certain point, but it's far from ideal)
>>>>>>>> There is also no exactly-once semantics. (updateStateByKey can
>>>>>>>> achieve this semantics, but is not practical if you have any 
>>>>>>>> significant
>>>>>>>> amount of state because it does so by dumping the entire state on every
>>>>>>>> checkpointing)
>>>>>>>>
>>>>>>>> There are also some minor drawbacks that I'm sure will be fixed
>>>>>>>> quickly, like no task timeout, not being able to read from Kafka using
>>>>>>>> multiple nodes, data loss hazard with Kafka.
>>>>>>>>
>>>>>>>> It's also not possible to attain very low latency in Spark, if
>>>>>>>> that's what you need.
>>>>>>>>
>>>>>>>> The pos for Spark is the concise and IMO more intuitive syntax,
>>>>>>>> especially if you compare it with Storm's Java API.
>>>>>>>>
>>>>>>>> I admit I might be a bit biased towards Storm tho as I'm more
>>>>>>>> familiar with it.
>>>>>>>>
>>>>>>>> Also, you can do some processing with Kinesis. If all you need to
>>>>>>>> do is straight forward transformation and you are reading from Kinesis 
>>>>>>>> to
>>>>>>>> begin with, it might be an easier option to just do the transformation 
>>>>>>>> in
>>>>>>>> Kinesis.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jun 17, 2015 at 7:15 AM, Sabarish Sasidharan <
>>>>>>>> [email protected]> wrote:
>>>>>>>>
>>>>>>>> Whatever you write in bolts would be the logic you want to apply on
>>>>>>>> your events. In Spark, that logic would be coded in map() or similar 
>>>>>>>> such
>>>>>>>> transformations and/or actions. Spark doesn't enforce a structure for
>>>>>>>> capturing your processing logic like Storm does.
>>>>>>>> Regards
>>>>>>>> Sab
>>>>>>>> Probably overloading the question a bit.
>>>>>>>>
>>>>>>>> In Storm, Bolts have the functionality of getting triggered on
>>>>>>>> events. Is that kind of functionality possible with Spark streaming? 
>>>>>>>> During
>>>>>>>> each phase of the data processing, the transformed data is stored to 
>>>>>>>> the
>>>>>>>> database and this transformed data should then be sent to a new 
>>>>>>>> pipeline
>>>>>>>> for further processing
>>>>>>>>
>>>>>>>> How can this be achieved using Spark?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jun 17, 2015 at 10:10 AM, Spark Enthusiast <
>>>>>>>> [email protected]> wrote:
>>>>>>>>
>>>>>>>> I have a use-case where a stream of Incoming events have to be
>>>>>>>> aggregated and joined to create Complex events. The aggregation will 
>>>>>>>> have
>>>>>>>> to happen at an interval of 1 minute (or less).
>>>>>>>>
>>>>>>>> The pipeline is :
>>>>>>>>                                   send events
>>>>>>>>                    enrich event
>>>>>>>> Upstream services -------------------> KAFKA ---------> event
>>>>>>>> Stream Processor ------------> Complex Event Processor ------------>
>>>>>>>> Elastic Search.
>>>>>>>>
>>>>>>>> From what I understand, Storm will make a very good ESP and Spark
>>>>>>>> Streaming will make a good CEP.
>>>>>>>>
>>>>>>>> But, we are also evaluating Storm with Trident.
>>>>>>>>
>>>>>>>> How does Spark Streaming compare with Storm with Trident?
>>>>>>>>
>>>>>>>> Sridhar Chellappa
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>   On Wednesday, 17 June 2015 10:02 AM, ayan guha <
>>>>>>>> [email protected]> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> I have a similar scenario where we need to bring data from kinesis
>>>>>>>> to hbase. Data volecity is 20k per 10 mins. Little manipulation of data
>>>>>>>> will be required but that's regardless of the tool so we will be 
>>>>>>>> writing
>>>>>>>> that piece in Java pojo.
>>>>>>>> All env is on aws. Hbase is on a long running EMR and kinesis on a
>>>>>>>> separate cluster.
>>>>>>>> TIA.
>>>>>>>> Best
>>>>>>>> Ayan
>>>>>>>> On 17 Jun 2015 12:13, "Will Briggs" <[email protected]> wrote:
>>>>>>>>
>>>>>>>> The programming models for the two frameworks are conceptually
>>>>>>>> rather different; I haven't worked with Storm for quite some time, but
>>>>>>>> based on my old experience with it, I would equate Spark Streaming more
>>>>>>>> with Storm's Trident API, rather than with the raw Bolt API. Even then,
>>>>>>>> there are significant differences, but it's a bit closer.
>>>>>>>>
>>>>>>>> If you can share your use case, we might be able to provide better
>>>>>>>> guidance.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Will
>>>>>>>>
>>>>>>>> On June 16, 2015, at 9:46 PM, [email protected] wrote:
>>>>>>>>
>>>>>>>> Hi All,
>>>>>>>>
>>>>>>>> I am evaluating spark VS storm ( spark streaming  ) and i am not
>>>>>>>> able to see what is equivalent of Bolt in storm inside spark.
>>>>>>>>
>>>>>>>> Any help will be appreciated on this ?
>>>>>>>>
>>>>>>>> Thanks ,
>>>>>>>> Ashish
>>>>>>>>
>>>>>>>> ---------------------------------------------------------------------
>>>>>>>> To unsubscribe, e-mail: [email protected]
>>>>>>>> For additional commands, e-mail: [email protected]
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> ---------------------------------------------------------------------
>>>>>>>> To unsubscribe, e-mail: [email protected]
>>>>>>>> For additional commands, e-mail: [email protected]
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to