asfgit closed pull request #7141: [FLINK-10763] [streaming] Fix interval join 
return type in Scala
URL: https://github.com/apache/flink/pull/7141
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
 
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index 580dd46d683..dc8df5035c0 100644
--- 
a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ 
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -201,7 +201,10 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) 
extends DataStream[T]
       * @return Returns a DataStream
       */
     @PublicEvolving
-    def process[OUT](processJoinFunction: ProcessJoinFunction[IN1, IN2, OUT]): 
DataStream[OUT] = {
+    def process[OUT: TypeInformation](
+        processJoinFunction: ProcessJoinFunction[IN1, IN2, OUT])
+      : DataStream[OUT] = {
+
       val javaJoined = new KeyedJavaStream.IntervalJoined[IN1, IN2, KEY](
         firstStream.javaStream.asInstanceOf[KeyedJavaStream[IN1, KEY]],
         secondStream.javaStream.asInstanceOf[KeyedJavaStream[IN2, KEY]],
@@ -209,7 +212,7 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) 
extends DataStream[T]
         upperBound,
         lowerBoundInclusive,
         upperBoundInclusive)
-      asScalaStream(javaJoined.process(processJoinFunction))
+      asScalaStream(javaJoined.process(processJoinFunction, 
implicitly[TypeInformation[OUT]]))
     }
   }
 
diff --git 
a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala
 
b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala
index 80701c6725a..a9ace36cc49 100644
--- 
a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala
+++ 
b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.scala
 
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction
 import org.apache.flink.streaming.api.functions.sink.SinkFunction
@@ -25,7 +26,8 @@ import 
org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExt
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.test.util.AbstractTestBase
 import org.apache.flink.util.Collector
-import org.junit.{Assert, Test}
+import org.junit.Assert.assertTrue
+import org.junit.Test
 
 import scala.collection.mutable.ListBuffer
 
@@ -47,22 +49,25 @@ class IntervalJoinITCase extends AbstractTestBase {
 
     val sink = new ResultSink()
 
-    dataStream1.intervalJoin(dataStream2)
+    val join = dataStream1.intervalJoin(dataStream2)
       .between(Time.milliseconds(0), Time.milliseconds(2))
-      .process(new CombineToStringJoinFunction())
-      .addSink(sink)
+      .process(new CombineJoinFunction())
+
+    assertTrue(join.dataType.isInstanceOf[CaseClassTypeInfo[_]])
+
+    join.addSink(sink)
 
     env.execute()
 
     sink.expectInAnyOrder(
-      "(key,0):(key,0)",
-      "(key,0):(key,1)",
-      "(key,0):(key,2)",
+      "(key:key,0)",
+      "(key:key,1)",
+      "(key:key,2)",
 
-      "(key,1):(key,1)",
-      "(key,1):(key,2)",
+      "(key:key,2)",
+      "(key:key,3)",
 
-      "(key,2):(key,2)"
+      "(key:key,4)"
     )
   }
 
@@ -82,18 +87,21 @@ class IntervalJoinITCase extends AbstractTestBase {
 
     val sink = new ResultSink()
 
-    dataStream1.intervalJoin(dataStream2)
+    val join = dataStream1.intervalJoin(dataStream2)
       .between(Time.milliseconds(0), Time.milliseconds(2))
       .lowerBoundExclusive()
       .upperBoundExclusive()
-      .process(new CombineToStringJoinFunction())
-      .addSink(sink)
+      .process(new CombineJoinFunction())
+
+    assertTrue(join.dataType.isInstanceOf[CaseClassTypeInfo[_]])
+
+    join.addSink(sink)
 
     env.execute()
 
     sink.expectInAnyOrder(
-      "(key,0):(key,1)",
-      "(key,1):(key,2)"
+      "(key:key,1)",
+      "(key:key,3)"
     )
   }
 }
@@ -102,14 +110,14 @@ object Companion {
   val results: ListBuffer[String] = new ListBuffer()
 }
 
-class ResultSink extends SinkFunction[String] {
+class ResultSink extends SinkFunction[(String, Long)] {
 
-  override def invoke(value: String, context: SinkFunction.Context[_]): Unit = 
{
-    Companion.results.append(value)
+  override def invoke(value: (String, Long), context: 
SinkFunction.Context[_]): Unit = {
+    Companion.results.append(value.toString())
   }
 
   def expectInAnyOrder(expected: String*): Unit = {
-    Assert.assertTrue(expected.toSet.equals(Companion.results.toSet))
+    assertTrue(expected.toSet.equals(Companion.results.toSet))
   }
 }
 
@@ -117,14 +125,14 @@ class TimestampExtractor extends 
AscendingTimestampExtractor[(String, Long)] {
   override def extractAscendingTimestamp(element: (String, Long)): Long = 
element._2
 }
 
-class CombineToStringJoinFunction
-  extends ProcessJoinFunction[(String, Long), (String, Long), String] {
+class CombineJoinFunction
+  extends ProcessJoinFunction[(String, Long), (String, Long), (String, Long)] {
 
   override def processElement(
-                        left: (String, Long),
-                        right: (String, Long),
-                        ctx: ProcessJoinFunction[(String, Long), (String, 
Long), String]#Context,
-                        out: Collector[String]): Unit = {
-    out.collect(left + ":" + right)
+      left: (String, Long),
+      right: (String, Long),
+      ctx: ProcessJoinFunction[(String, Long), (String, Long), (String, 
Long)]#Context,
+      out: Collector[(String, Long)]): Unit = {
+    out.collect((left._1 + ":" + right._1, left._2 + right._2))
   }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to