dawidwys closed pull request #7168: [FLINK-6756][DataStream API] Provide Rich
AsyncFunction to Scala API …
URL: https://github.com/apache/flink/pull/7168
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/AsyncDataStream.scala
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
index a1568c29b61..804acfcf045 100644
---
a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
+++
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
@@ -21,9 +21,8 @@ package org.apache.flink.streaming.api.scala
import org.apache.flink.annotation.PublicEvolving
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{AsyncDataStream =>
JavaAsyncDataStream}
-import org.apache.flink.streaming.api.functions.async.{ResultFuture =>
JavaResultFuture}
-import org.apache.flink.streaming.api.functions.async.{AsyncFunction =>
JavaAsyncFunction}
-import org.apache.flink.streaming.api.scala.async.{AsyncFunction,
JavaResultFutureWrapper, ResultFuture}
+import org.apache.flink.streaming.api.functions.async.{AsyncFunction =>
JavaAsyncFunction, ResultFuture => JavaResultFuture}
+import org.apache.flink.streaming.api.scala.async._
import org.apache.flink.util.Preconditions
import scala.concurrent.duration.TimeUnit
@@ -67,14 +66,7 @@ object AsyncDataStream {
capacity: Int)
: DataStream[OUT] = {
- val javaAsyncFunction = new JavaAsyncFunction[IN, OUT] {
- override def asyncInvoke(input: IN, resultFuture:
JavaResultFuture[OUT]): Unit = {
- asyncFunction.asyncInvoke(input, new
JavaResultFutureWrapper(resultFuture))
- }
- override def timeout(input: IN, resultFuture: JavaResultFuture[OUT]):
Unit = {
- asyncFunction.timeout(input, new JavaResultFutureWrapper(resultFuture))
- }
- }
+ val javaAsyncFunction = wrapAsJavaAsyncFunction(asyncFunction)
val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
@@ -197,14 +189,7 @@ object AsyncDataStream {
capacity: Int)
: DataStream[OUT] = {
- val javaAsyncFunction = new JavaAsyncFunction[IN, OUT] {
- override def asyncInvoke(input: IN, resultFuture:
JavaResultFuture[OUT]): Unit = {
- asyncFunction.asyncInvoke(input, new
JavaResultFutureWrapper[OUT](resultFuture))
- }
- override def timeout(input: IN, resultFuture: JavaResultFuture[OUT]):
Unit = {
- asyncFunction.timeout(input, new
JavaResultFutureWrapper[OUT](resultFuture))
- }
- }
+ val javaAsyncFunction = wrapAsJavaAsyncFunction(asyncFunction)
val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
@@ -229,12 +214,11 @@ object AsyncDataStream {
* @return the resulting stream containing the asynchronous results
*/
def orderedWait[IN, OUT: TypeInformation](
- input: DataStream[IN],
- asyncFunction: AsyncFunction[IN, OUT],
- timeout: Long,
- timeUnit: TimeUnit)
- : DataStream[OUT] = {
-
+ input: DataStream[IN],
+ asyncFunction: AsyncFunction[IN, OUT],
+ timeout: Long,
+ timeUnit: TimeUnit)
+ : DataStream[OUT] = {
orderedWait(input, asyncFunction, timeout, timeUnit,
DEFAULT_QUEUE_CAPACITY)
}
@@ -253,12 +237,12 @@ object AsyncDataStream {
* @return the resulting stream containing the asynchronous results
*/
def orderedWait[IN, OUT: TypeInformation](
- input: DataStream[IN],
- timeout: Long,
- timeUnit: TimeUnit,
- capacity: Int) (
- asyncFunction: (IN, ResultFuture[OUT]) => Unit)
- : DataStream[OUT] = {
+ input: DataStream[IN],
+ timeout: Long,
+ timeUnit: TimeUnit,
+ capacity: Int) (
+ asyncFunction: (IN, ResultFuture[OUT]) => Unit)
+ : DataStream[OUT] = {
Preconditions.checkNotNull(asyncFunction)
@@ -293,12 +277,29 @@ object AsyncDataStream {
* @return the resulting stream containing the asynchronous results
*/
def orderedWait[IN, OUT: TypeInformation](
- input: DataStream[IN],
- timeout: Long,
- timeUnit: TimeUnit) (
- asyncFunction: (IN, ResultFuture[OUT]) => Unit)
- : DataStream[OUT] = {
+ input: DataStream[IN],
+ timeout: Long,
+ timeUnit: TimeUnit) (
+ asyncFunction: (IN, ResultFuture[OUT]) => Unit)
+ : DataStream[OUT] = {
orderedWait(input, timeout, timeUnit,
DEFAULT_QUEUE_CAPACITY)(asyncFunction)
}
+
+ private def wrapAsJavaAsyncFunction[IN, OUT: TypeInformation](
+ asyncFunction: AsyncFunction[IN, OUT])
+ : JavaAsyncFunction[IN, OUT] = asyncFunction match {
+ case richAsyncFunction: RichAsyncFunction[IN, OUT] =>
+ new ScalaRichAsyncFunctionWrapper[IN, OUT](richAsyncFunction)
+ case _ =>
+ new JavaAsyncFunction[IN, OUT] {
+ override def asyncInvoke(input: IN, resultFuture:
JavaResultFuture[OUT]): Unit = {
+ asyncFunction.asyncInvoke(input, new
JavaResultFutureWrapper[OUT](resultFuture))
+ }
+
+ override def timeout(input: IN, resultFuture: JavaResultFuture[OUT]):
Unit = {
+ asyncFunction.timeout(input, new
JavaResultFutureWrapper[OUT](resultFuture))
+ }
+ }
+ }
}
diff --git
a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/RichAsyncFunction.scala
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/RichAsyncFunction.scala
new file mode 100644
index 00000000000..e6526251de6
--- /dev/null
+++
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/RichAsyncFunction.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.scala.async
+
+import org.apache.flink.api.common.functions.AbstractRichFunction
+
+/**
+ * Rich variant of [[AsyncFunction]]. As a
[[org.apache.flink.api.common.functions.RichFunction]],
+ * it gives access to the
[[org.apache.flink.api.common.functions.RuntimeContext]] and provides
+ * setup and teardown methods.
+ *
+ * State related apis in
[[org.apache.flink.api.common.functions.RuntimeContext]] are not supported
+ * yet because the key may get changed while accessing states in the working
thread.
+ *
+ *
[[org.apache.flink.api.common.functions.IterationRuntimeContext#getIterationAggregator(String)]]
+ * is not supported since the aggregator may be modified by multiple threads.
+ *
+ * @tparam IN The type of the input value.
+ * @tparam OUT The type of the output value.
+ */
+abstract class RichAsyncFunction[IN, OUT]
+ extends AbstractRichFunction
+ with AsyncFunction [IN, OUT] {}
diff --git
a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala
new file mode 100644
index 00000000000..2dd3f323910
--- /dev/null
+++
b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.scala.async
+
+import org.apache.flink.api.common.functions.RuntimeContext
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.async.{ResultFuture =>
JResultFuture, RichAsyncFunction => JRichAsyncFunction}
+
+/**
+ * A wrapper function that exposes a Scala RichAsyncFunction as a Java Rich
Async Function.
+ *
+ * The Scala and Java RichAsyncFunctions differ in their type of
"ResultFuture"
+ * - Scala RichAsyncFunction:
org.apache.flink.streaming.api.scala.async.ResultFuture
+ * - Java RichAsyncFunction:
org.apache.flink.streaming.api.functions.async.ResultFuture
+ */
+final class ScalaRichAsyncFunctionWrapper[IN, OUT](func: RichAsyncFunction[IN,
OUT])
+ extends JRichAsyncFunction[IN, OUT]{
+
+ override def asyncInvoke(input: IN, resultFuture: JResultFuture[OUT]): Unit
= {
+ func.asyncInvoke(input, new JavaResultFutureWrapper[OUT](resultFuture))
+ }
+
+ override def timeout(input: IN, resultFuture: JResultFuture[OUT]): Unit = {
+ func.timeout(input, new JavaResultFutureWrapper[OUT](resultFuture))
+ }
+
+ override def open(parameters: Configuration): Unit = {
+ func.open(parameters)
+ }
+
+ override def close(): Unit = {
+ func.close()
+ }
+
+ override def setRuntimeContext(runtimeContext: RuntimeContext): Unit = {
+ super.setRuntimeContext(runtimeContext)
+ func.setRuntimeContext(super.getRuntimeContext)
+ }
+}
diff --git
a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala
b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala
index d0a2cec9a4f..653c982af9e 100644
---
a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala
+++
b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala
@@ -20,9 +20,11 @@ package org.apache.flink.streaming.api.scala
import java.util.concurrent.TimeUnit
+import org.apache.flink.configuration.Configuration
+import
org.apache.flink.streaming.api.functions.async.RichAsyncFunction.{RichAsyncFunctionIterationRuntimeContext,
RichAsyncFunctionRuntimeContext}
import org.apache.flink.streaming.api.functions.sink.SinkFunction
import org.apache.flink.streaming.api.scala.AsyncDataStreamITCase._
-import org.apache.flink.streaming.api.scala.async.{AsyncFunction, ResultFuture}
+import org.apache.flink.streaming.api.scala.async.{AsyncFunction,
ResultFuture, RichAsyncFunction}
import org.apache.flink.test.util.AbstractTestBase
import org.junit.Assert._
import org.junit.Test
@@ -95,6 +97,21 @@ class AsyncDataStreamITCase extends AbstractTestBase {
testAsyncWaitUsingAnonymousFunction(false)
}
+ @Test
+ def testRichAsyncFunctionRuntimeContext(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setParallelism(1)
+
+ val source = env.fromElements(1)
+
+
+ val richAsyncFunction = new MyRichAsyncFunction
+ val asyncMapped = AsyncDataStream
+ .unorderedWait(source, richAsyncFunction, timeout, TimeUnit.MILLISECONDS)
+
+ executeAndValidate(false, env, asyncMapped, mutable.ArrayBuffer[Int](2))
+ }
+
private def testAsyncWaitUsingAnonymousFunction(ordered: Boolean): Unit = {
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setParallelism(1)
@@ -135,3 +152,24 @@ class MyAsyncFunction extends AsyncFunction[Int, Int] {
resultFuture.complete(Seq(input * 3))
}
}
+
+class MyRichAsyncFunction extends RichAsyncFunction[Int, Int] {
+
+ override def open(parameters: Configuration): Unit = {
+ assertEquals(getRuntimeContext.getNumberOfParallelSubtasks, 1)
+ }
+
+ override def asyncInvoke(input: Int, resultFuture: ResultFuture[Int]): Unit
= {
+ Future {
+ // trigger the timeout of the even input number
+ if (input % 2 == 0) {
+ Thread.sleep(AsyncDataStreamITCase.timeout + 1000)
+ }
+
+ resultFuture.complete(Seq(input * 2))
+ } (ExecutionContext.global)
+ }
+ override def timeout(input: Int, resultFuture: ResultFuture[Int]): Unit = {
+ resultFuture.complete(Seq(input * 3))
+ }
+}
----------------------------------------------------------------
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