Hi Stephan,

I'm skeptical about two things:
- using processing time will result in inaccurately bounded aggregates (or
do you want to group by event time in a processing time window?)
- writing to and reading from Cassandra might be expensive (not sure what
you mean by cheaper in the end) and it is not integrated with Flink's
checkpointing mechanism for fault-tolerance.

To me, the stateful FlatMapOperator looks like the best approach. There is
an upcoming feature for registering timers in user-functions, i.e., a
function is called after the timer exceeds. This could be helpful to
overcome the problem of closing the window without new data.

Best,
Fabian

2016-11-14 8:39 GMT+01:00 Stephan Epping <stephan.epp...@zweitag.de>:

> Hello Fabian,
>
> Thank you very much. What is your opinion on the following solution:
>
> - Window data per time window, e.g. 15 minutes
> - using processing time as trigger, e.g. 15 minutes
> - which results in an aggregate over sensor values
> - then use cassandra to select the previous aggregate (as there can be
> multiple for the time window due to processing time)
> - then update the aggregate and put it into a cassandra sink again
>
> The cassandra select will be a bit slower than using an in memory/flink
> state, but will be cheaper in the end. Further, what does this have for
> consequences?
> For example, replaying events will be more difficult, right? Also, what
> about Snapshots? Will they work with the mentioned design?
>
> kind regards,
> Stephan
>
>
> On 11 Nov 2016, at 00:39, Fabian Hueske <fhue...@gmail.com> wrote:
>
> Hi Stephan,
>
> I just wrote an answer to your SO question.
>
> Best, Fabian
>
> 2016-11-10 11:01 GMT+01:00 Stephan Epping <stephan.epp...@zweitag.de>:
>
>> Hello,
>>
>> I found this question in the Nabble archive (
>> http://apache-flink-user-mailing-list-archive.2336050.n4.
>> nabble.com/Maintaining-watermarks-per-key-instead-of-per-
>> operator-instance-tp7288.html) but was unable/dont know how to reply.
>>
>> Here is my question regarding the mentioned thread:
>>
>> Hello,
>>
>> I have similar requirements (see StackOverflor http://stac
>> koverflow.com/questions/40465335/apache-flink-multiple-
>> window-aggregations-and-late-data). I am pretty new to flink, could you
>> elaborate on a possible solution? We can guarantee good ordering by
>> sensor_id, thus watermarking by key would be the only reasonable way for us
>> (*sensorData.keyBy('id').timeWindow(1.minute).sum('value')*), could I do
>> my own watermarking after*sensorData.keyBy('id').overwriteWatermarking()*...
>> per key? Or maybe using custom state plus a custom trigger? What happens if
>> a sensor dies or is being removed completely, how can this be detected as
>> watermarks would be ignored for window garbage collection. Or could we
>> dynamically schedule a job of each sensor? Which would result in 1000 Jobs.
>>
>>
>> Thanks,
>> Stephan
>>
>>
>>
>
>

Reply via email to