You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by dw...@apache.org on 2021/05/05 19:10:38 UTC

[flink] branch master updated: [FLINK-22442][CEP] Using scala api to change the TimeCharacteristic of the PatternStream is invalid

This is an automated email from the ASF dual-hosted git repository.

dwysakowicz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new cb0d8eb  [FLINK-22442][CEP] Using scala api to change the TimeCharacteristic of the PatternStream is invalid
cb0d8eb is described below

commit cb0d8eb85048cfb08be54e75f2f90b1d590b7ae0
Author: chennuo <ch...@didachuxing.com>
AuthorDate: Sat Apr 24 21:39:28 2021 +0800

    [FLINK-22442][CEP] Using scala api to change the TimeCharacteristic of the PatternStream is invalid
    
    This closes #15742
---
 .../main/scala/org/apache/flink/cep/scala/PatternStream.scala | 11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)

diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala
index 93c4d30..ebc1c27 100644
--- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala
+++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala
@@ -447,18 +447,15 @@ class PatternStream[T](jPatternStream: JPatternStream[T]) {
   }
 
  def sideOutputLateData(lateDataOutputTag: OutputTag[T]): PatternStream[T] = {
-   jPatternStream.sideOutputLateData(lateDataOutputTag)
-   this
- }
+    PatternStream[T](jPatternStream.sideOutputLateData(lateDataOutputTag))
+  }
 
   def inProcessingTime(): PatternStream[T] = {
-    jPatternStream.inProcessingTime()
-    this
+    PatternStream[T](jPatternStream.inProcessingTime())
   }
 
   def inEventTime(): PatternStream[T] = {
-    jPatternStream.inEventTime()
-    this
+    PatternStream[T](jPatternStream.inEventTime())
   }
 }