Re: Stateful streaming question

2017-06-16 Thread Aljoscha Krettek
t;> >>>> >>>> >>>> 1) Why is current batch mode expensive? Where are you persisting the >>>> data after updates? Way I see it by moving to Flink, you get to use >>>> RocksDB(a key-value store) that makes your lookup

Re: Stateful streaming question

2017-06-16 Thread Flavio Pompermaier
a after updates? Way I see it by moving to Flink, you get to use >>>> RocksDB(a key-value store) that makes your lookups faster – probably right >>>> now you are using a non-indexed store like S3 maybe? >>>> >>>> So, gain is coming from moving to

Re: Stateful streaming question

2017-06-16 Thread Aljoscha Krettek
than from batch->streaming. Myabe consider just going with a >>> different data store. >>> >>> IMHO, stream should only be used if you really want to act on the new >>> events in real-time. It is generally harder to get a streaming job correct >>> than a

Re: Stateful streaming question

2017-06-15 Thread Flavio Pompermaier
ne. >>> >>> >>> >>> 2) If current setup is expensive due to >>> serialization-deserialization then that should be fixed by moving to a >>> faster format (maybe AVRO? - I don’t have a lot of expertise in that). I >>> don’t see how that pr

Re: Stateful streaming question

2017-06-15 Thread Aljoscha Krettek
ster format (maybe AVRO? - I >> don’t have a lot of expertise in that). I don’t see how that problem will go >> away with Flink – so still need to handle serialization. >> >> >> >> 3) Even if you do decide to move to Flink – I think you can do this

Re: Stateful streaming question

2017-05-17 Thread Flavio Pompermaier
pertise in that). I >> don’t see how that problem will go away with Flink – so still need to >> handle serialization. >> >> >> >> 3) Even if you do decide to move to Flink – I think you can do >> this with one job, two jobs are not needed. At every incoming event, c

Re: Stateful streaming question

2017-05-17 Thread Kostas Kloudas
needed. At every incoming event, check the > previous state and update/output to kafka or whatever data store you are > using. > > > > > > Thanks > > Ankit > > > > From: Flavio Pompermaier mailto:pomperma...@okkam.it>> > Date: Tuesda

Re: Stateful streaming question

2017-05-17 Thread Fabian Hueske
n if you do decide to move to Flink – I think you can do this > with one job, two jobs are not needed. At every incoming event, check the > previous state and update/output to kafka or whatever data store you are > using. > > > > > > Thanks > > Ankit > > > >

Re: Stateful streaming question

2017-05-16 Thread Jain, Ankit
ate/output to kafka or whatever data store you are using. Thanks Ankit From: Flavio Pompermaier Date: Tuesday, May 16, 2017 at 9:31 AM To: Kostas Kloudas Cc: user Subject: Re: Stateful streaming question Hi Kostas, thanks for your quick response. I also thought about using Async IO, I just need

Re: Stateful streaming question

2017-05-16 Thread Flavio Pompermaier
Hi Kostas, thanks for your quick response. I also thought about using Async IO, I just need to figure out how to correctly handle parallelism and number of async requests. However that's probably the way to go..is it possible also to set a number of retry attempts/backoff when the async request fai

Re: Stateful streaming question

2017-05-16 Thread Kostas Kloudas
Hi Flavio, From what I understand, for the first part you are correct. You can use Flink’s internal state to keep your enriched data. In fact, if you are also querying an external system to enrich your data, it is worth looking at the AsyncIO feature: https://ci.apache.org/projects/flink/flink-

Stateful streaming question

2017-05-16 Thread Flavio Pompermaier
Hi to all, we're still playing with Flink streaming part in order to see whether it can improve our current batch pipeline. At the moment, we have a job that translate incoming data (as Row) into Tuple4, groups them together by the first field and persist the result to disk (using a thrift object).