[ https://issues.apache.org/jira/browse/FLINK-3682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15217969#comment-15217969 ]
ASF GitHub Bot commented on FLINK-3682: --------------------------------------- GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/1841 [FLINK-3682] [cep] Assign processing timestamp in CEP operators This PR fixes the problem that the CEP operators did not assign the wall clock time as the timestamp to incoming in StreamRecords if the TimeCharacteristic was set to ProcessingTime. Processing element with a Long.MIN_VALUE timestamp can lead to underflows in the NFA if a positive window length is subtracted from the timestamp. For this underflow a sanity check has been added to notify the user with an exception about it. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink fixUnderflowPruning Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/1841.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1841 ---- commit 3adc65e0d65604b6255f792f00a76ec017170f0d Author: Till Rohrmann <trohrm...@apache.org> Date: 2016-03-30T13:27:21Z [FLINK-3682] [cep] Assign processing timestamp in CEP operators This PR fixes the problem that the CEP operators did not assign the wall clock time as the timestamp to incoming in StreamRecords if the TimeCharacteristic was set to ProcessingTime. Processing element with a Long.MIN_VALUE timestamp can lead to underflows in the NFA if a positive window length is subtracted from the timestamp. For this underflow a sanity check has been added to notify the user with an exception about it. ---- > CEP operator does not set the processing timestamp correctly > ------------------------------------------------------------ > > Key: FLINK-3682 > URL: https://issues.apache.org/jira/browse/FLINK-3682 > Project: Flink > Issue Type: Bug > Components: CEP > Affects Versions: 1.0.0 > Reporter: Till Rohrmann > Assignee: Till Rohrmann > Fix For: 1.1.0, 1.0.1 > > > In the wake of reworking the timestamp assignment where the processing > timestamp has to be set now by the {{StreamOperator}}, the CEP operators have > not been adapted. This causes that the timestamp value assigned to the > {{StreamRecord}} is used. In case of processing time this is > {{Long.MIN_VALUE}}. In combination with a CEP time window, this can lead to > an underflow in the NFA where the window time is subtracted from the current > timestamp value. -- This message was sent by Atlassian JIRA (v6.3.4#6332)