Thank you, Kostas, for reviewing this.
Although points 1 and 3 are something which I was planning to address in
the actual implementation, #2 would still be a show stopper.
I'll spend some more time on this and maybe come up with a better way to
achieve the same use case without mixing the two no
Hi Shailesh,
Your solution may fit your use case, but as Dawid mentioned earlier, it makes a
lot of
assumptions about the input.
From a look at your PoC:
1) You assume no late data (you do not drop anything) and no out-of-orderness.
2) You mix the two notions of time (event and processing).
3)
Bump.
On Thu, Mar 22, 2018 at 7:54 PM, Shailesh Jain
wrote:
> To trigger the computations for each batch, I'll have to use the
> processing time timer in the abstract keyed cep operator, right?
>
> The reason why I'm avoiding the watermarks is that it is not possible to
> generate watermarks per
To trigger the computations for each batch, I'll have to use the processing
time timer in the abstract keyed cep operator, right?
The reason why I'm avoiding the watermarks is that it is not possible to
generate watermarks per key.
Thanks for the 'within' remark.
A couple of questions:
1. Given
If you do the buffering you can emit watermark for each such batch (equal to
highest timestamp in such batch). This way you won’t need to sort. CEP library
will do it for you.
The within clause will work in EventTime then.
One more remark also the within clause always work for whole pattern not
Thanks for your reply, Dawid.
I understand that the approach I've tried out is not generic enough, and
would need a lot more thought to be put into w.r.t parallelism
considerations, out of order events, effects on downstream operators etc.
The intention was to do a quick implementation to check th
Hi Shailesh,
Thanks for your interest in the CEP library and sorry for late response. I must
say I am not fun of this approach.
After this change, the Processing time is no longer a processing time, plus it
will work differently in any other place of Flink. It will also not sort the
events etc.
Thanks Aljoscha.
Bump.
I understand everyone would be busy with 1.5.0, but would really appreciate
slight help in unblocking us here.
Thanks,
Shailesh
On Thu, Mar 15, 2018 at 1:47 AM, Aljoscha Krettek
wrote:
> Hi,
>
> I think this should have been sent to the dev mailing list because in the
>
Hi,
I think this should have been sent to the dev mailing list because in the user
mailing list it might disappear among a lot of other mail.
Forwarding...
Best,
Aljoscha
> On 14. Mar 2018, at 06:20, Shailesh Jain wrote:
>
> Hi,
>
> We've been facing issues* w.r.t watermarks not supported p