[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3644 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3644#discussion_r108958127 --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java --- @@ -196,19 +213,45 @@ public void processElement(StreamRecord element) throws Exception { updateNFA(nfa); } else { - getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); - PriorityQueuepriorityQueue = getPriorityQueue(); + // In event-time processing we assume correctness of the watermark. + // Events with timestamp smaller than the last seen watermark are considered late. + // Late events are put in a dedicated side output, if the user has specified one. + + if (element.getTimestamp() >= lastWatermark) { - // event time processing - // we have to buffer the elements until we receive the proper watermark - if (getExecutionConfig().isObjectReuseEnabled()) { - // copy the StreamRecord so that it cannot be changed - priorityQueue.offer(new StreamRecord(inputSerializer.copy(element.getValue()), element.getTimestamp())); + // we have an event with a valid timestamp, so + // we buffer it until we receive the proper watermark. + + getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); + + PriorityQueue priorityQueue = getPriorityQueue(); + if (getExecutionConfig().isObjectReuseEnabled()) { + // copy the StreamRecord so that it cannot be changed + priorityQueue.offer(new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp())); + } else { + priorityQueue.offer(element); + } + updatePriorityQueue(priorityQueue); } else { - priorityQueue.offer(element); + sideOutputLateElement(element); } - updatePriorityQueue(priorityQueue); + } + } + + private void updateLastSeenWatermark(Watermark watermark) { + this.lastWatermark = watermark.getTimestamp(); --- End diff -- I believe that the `Watermark` handling should be the responsibility of the sources. Checkpointing it here would mean special logic for handling it in case of scaling down for example. In addition, when scaling down e.g from 2 tasks to 1 and the 2 tasks had different last watermarks, then we still may have issues with elements being processed out-of-order. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
Github user dawidwys commented on a diff in the pull request: https://github.com/apache/flink/pull/3644#discussion_r108866698 --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java --- @@ -196,19 +213,45 @@ public void processElement(StreamRecord element) throws Exception { updateNFA(nfa); } else { - getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); - PriorityQueuepriorityQueue = getPriorityQueue(); + // In event-time processing we assume correctness of the watermark. + // Events with timestamp smaller than the last seen watermark are considered late. + // Late events are put in a dedicated side output, if the user has specified one. + + if (element.getTimestamp() >= lastWatermark) { - // event time processing - // we have to buffer the elements until we receive the proper watermark - if (getExecutionConfig().isObjectReuseEnabled()) { - // copy the StreamRecord so that it cannot be changed - priorityQueue.offer(new StreamRecord(inputSerializer.copy(element.getValue()), element.getTimestamp())); + // we have an event with a valid timestamp, so + // we buffer it until we receive the proper watermark. + + getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); + + PriorityQueue priorityQueue = getPriorityQueue(); + if (getExecutionConfig().isObjectReuseEnabled()) { + // copy the StreamRecord so that it cannot be changed + priorityQueue.offer(new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp())); + } else { + priorityQueue.offer(element); + } + updatePriorityQueue(priorityQueue); } else { - priorityQueue.offer(element); + sideOutputLateElement(element); } - updatePriorityQueue(priorityQueue); + } + } + + private void updateLastSeenWatermark(Watermark watermark) { + this.lastWatermark = watermark.getTimestamp(); --- End diff -- Anyway would be nice to see a testcase like watermark(10) restoreFromCheckpoint event(7) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
Github user dawidwys commented on a diff in the pull request: https://github.com/apache/flink/pull/3644#discussion_r108741567 --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java --- @@ -196,19 +213,45 @@ public void processElement(StreamRecord element) throws Exception { updateNFA(nfa); } else { - getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); - PriorityQueuepriorityQueue = getPriorityQueue(); + // In event-time processing we assume correctness of the watermark. + // Events with timestamp smaller than the last seen watermark are considered late. + // Late events are put in a dedicated side output, if the user has specified one. + + if (element.getTimestamp() >= lastWatermark) { - // event time processing - // we have to buffer the elements until we receive the proper watermark - if (getExecutionConfig().isObjectReuseEnabled()) { - // copy the StreamRecord so that it cannot be changed - priorityQueue.offer(new StreamRecord(inputSerializer.copy(element.getValue()), element.getTimestamp())); + // we have an event with a valid timestamp, so + // we buffer it until we receive the proper watermark. + + getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue())); + + PriorityQueue priorityQueue = getPriorityQueue(); + if (getExecutionConfig().isObjectReuseEnabled()) { + // copy the StreamRecord so that it cannot be changed + priorityQueue.offer(new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp())); + } else { + priorityQueue.offer(element); + } + updatePriorityQueue(priorityQueue); } else { - priorityQueue.offer(element); + sideOutputLateElement(element); } - updatePriorityQueue(priorityQueue); + } + } + + private void updateLastSeenWatermark(Watermark watermark) { + this.lastWatermark = watermark.getTimestamp(); --- End diff -- Shouldn't the lastWatermark be stored in a StateBackend? How will it behave after restoring from a checkpoint? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
Github user dawidwys commented on a diff in the pull request: https://github.com/apache/flink/pull/3644#discussion_r108742662 --- Diff: docs/dev/libs/cep.md --- @@ -711,6 +711,57 @@ DataStream[Either[TimeoutEvent, ComplexEvent]] result = patternStream.flatSelect +### Handling Lateness in Event Time + +In `CEP` the order in which elements are processed matters. To this end and when working in event time, an incoming --- End diff -- I don't get the first part of the sentence : "To this end and when working in event time" --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #3644: [FLINK-6205] [FLINK-6069] [cep] Correct watermark/...
GitHub user kl0u opened a pull request: https://github.com/apache/flink/pull/3644 [FLINK-6205] [FLINK-6069] [cep] Correct watermark/late events in side-output With this, the CEP library assumes correctness of the watermark and considers as late, events that arrive having a timestamp smaller than that of the last seen watermark. Late events are not silently dropped, but the user can specify to send them to a side output. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kl0u/flink late-element-cep Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3644.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 #3644 commit 827bd85916206e36ed23564e77e5ff119bfadead Author: kl0uDate: 2017-03-23T18:01:15Z [FLINK-6205] [FLINK-6069] [cep] Correct watermark/late events in side output. With this, the CEP library assumes correctness of the watermark and considers as late, events that arrive having a timestamp smaller than that of the last seen watermark. Late events are not silently dropped, but the user can specify to send them to a side output. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---