[
https://issues.apache.org/jira/browse/FLINK-5753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16006685#comment-16006685
]
Kostas Kloudas commented on FLINK-5753:
---------------------------------------
Hi [~jurijuri]! Thanks for reporting this!
I assume that your job is operating in {{processingTime}} right?
In this case, this is (unfortunately) expected to happen as in processing time,
it is events only that trigger computations and not timers.
The reason is that contrary to event time, where watermarks define a coarser
time granularity, on event time there is not such thing, so we use event
arrival to regulate the
frequency processing happens. If it were to register timers in processing time
that will trigger computation when they fire, then:
1) we would have to introduce another parameter (sth like timer interval) that
would imitate the watermark
2) we would have to register a timer for each incoming element (which can lead
to a
significant increase in storage requirements)
Currently we also do it for event time timers, but for event time we have ideas
on how to change it for upcoming releases.
Is this a blocker for your usecase, or you just worked in processing time for
debugging and your usecase uses processing time?
> CEP timeout handler.
> --------------------
>
> Key: FLINK-5753
> URL: https://issues.apache.org/jira/browse/FLINK-5753
> Project: Flink
> Issue Type: Bug
> Components: CEP
> Affects Versions: 1.1.2
> Reporter: MichaĆ Jurkiewicz
>
> I configured the following flink job in my environment:
> {code}
> Pattern<Event, ?> patternCommandStarted = Pattern.<Event>
> begin("event-accepted").subtype(Event.class)
> .where(e -> {event accepted where
> statement}).next("second-event-started").subtype(Event.class)
> .where(e -> {event started where statement}))
> .within(Time.seconds(30));
> DataStream<Either<Event, Event>> events = CEP
> .pattern(eventsStream.keyBy(e -> e.getEventProperties().get("deviceCode")),
> patternCommandStarted)
> .select(eventSelector, eventSelector);
> static class EventSelector implements PatternSelectFunction<Event, Event>,
> PatternTimeoutFunction<Event, Event> {}
> {code}
> The problem that I have is related to timeout handling. I observed that:
> if: first event appears, second event not appear in the stream
> and *no new events appear in a stream*, timeout handler is not executed.
> Expected result: timeout handler should be executed in case if there are no
> new events in a stream
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)