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.
Moreover I think you could achieve pretty similar solution if you generate your 
watermark based on the machine time. If in the getCurrentWatermark method
of your AssignerWithPeriodicWatermarks you will just return new 
Watermark(System.currentTimeMillis()), you will get the same behaviour as with 
that change, am I right?

Best,
Dawid

> On 18 Mar 2018, at 09:00, Shailesh Jain <shailesh.j...@stellapps.com> wrote:
> 
> 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 <aljos...@apache.org>
> wrote:
> 
>> 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 <shailesh.j...@stellapps.com>
>> wrote:
>>> 
>>> Hi,
>>> 
>>> We've been facing issues* w.r.t watermarks not supported per key, which
>> led us to:
>>> 
>>> Either (a) run the job in Processing time for a KeyedStream ->
>> compromising on use cases which revolve around catching time-based patterns
>>> or (b) run the job in Event time for multiple data streams (one data
>> stream per key) -> this is not scalable as the number of operators grow
>> linearly with the number of keys
>>> 
>>> To address this, we've done a quick (poc) change in the
>> AbstractKeyedCEPPatternOperator to allow for the NFAs to progress based
>> on timestamps extracted from the events arriving into the operator (and not
>> from the watermarks). We've tested it against our usecase and are seeing a
>> significant improvement in memory usage without compromising on the
>> watermark functionality.
>>> 
>>> It'll be really helpful if someone from the cep dev group can take a
>> look at this branch - https://github.com/jainshailesh/flink/commits/
>> cep_changes <https://github.com/jainshailesh/flink/commits/cep_changes>
>> and provide comments on the approach taken, and maybe guide us on the next
>> steps for taking it forward.
>>> 
>>> Thanks,
>>> Shailesh
>>> 
>>> * Links to previous email threads related to the same issue:
>>> http://apache-flink-user-mailing-list-archive.2336050.
>> n4.nabble.com/Question-on-event-time-functionality-
>> using-Flink-in-a-IoT-usecase-td18653.html <http://apache-flink-user-
>> mailing-list-archive.2336050.n4.nabble.com/Question-on-
>> event-time-functionality-using-Flink-in-a-IoT-usecase-td18653.html>
>>> http://apache-flink-user-mailing-list-archive.2336050.
>> n4.nabble.com/Generate-watermarks-per-key-in-a-KeyedStream-td16629.html <
>> http://apache-flink-user-mailing-list-archive.2336050.
>> n4.nabble.com/Generate-watermarks-per-key-in-a-KeyedStream-td16629.html>
>>> http://apache-flink-user-mailing-list-archive.2336050.
>> n4.nabble.com/Correlation-between-number-of-operators-
>> and-Job-manager-memory-requirements-td18384.html <
>> http://apache-flink-user-mailing-list-archive.2336050.
>> n4.nabble.com/Correlation-between-number-of-operators-
>> and-Job-manager-memory-requirements-td18384.html>
>>> 
>> 
>> 

Attachment: signature.asc
Description: Message signed with OpenPGP

Reply via email to