From 2366df9390807abd0f44bedb105b087d7cdc90e2 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Wed, 24 Feb 2016 13:05:16 +0100 Subject: [PATCH 01/39] [FLINK-1159] Case style anonymous functions not supported by Scala API [FLINK-1159] ClassTag context bound remove, tests added, new package layout implemented [FLINK-1159] OnKeyedStream.foldWith implemented [FLINK-1159] Licenses added, class stubs created [FLINK-1159] Added missing licenses [FLINK-1159] OnGroupedDataSet implemented [FLINK-1159] OnCrossDataSet implemented [FLINK-1159] Implemented OnConnectedStream [FLINK-1159] Methods in OnGroupedDataSet renamed to preserve name consistency [FLINK-1159] Implemented OnWindowedStream [FLINK-1159] Improved ScalaDoc [FLINK-1159] Add implicit conversions for newly implemented types [FLINK-1159] Several fixes - Restore required ClassTag contexts - Fix syntax error in implicit conversion for WindowedStream - Add missing implicit conversions for implemented method [FLINK-1159] Add missing newline character at the end of files [FLINK-1159] Add Scala API extensions page (stub) [FLINK-1159] Add Scala API extensions reference to batch APIs guide [FLINK-1159] Add Scala API extensions reference to streaming APIs guide [FLINK-1159] Parameters curried and inverted for sortGroupWith method [FLINK-1159] Accept partial functions documented (first draft) [FLINK-1159] Solve compilation issue with OnJoinedStream and OnWindowedStream [FLINK-1159] Fixed style issues with OnWindowedStream [FLINK-1159] Add missing newlines at end of ITs [FLINK-1159] Reword a couple of sentences [FLINK-1159] Reword: 'extension' to 'extension methods' [FLINK-1159] Introduce more functional interface to applyWith method [FLINK-1159] Remove unused import [FLINK-1159] Add stubs for missing methods in docs [FLINK-1159] Apply currying to follow the master, rename function parameters [FLINK-1159] Added missing examples [FLINK-1159] Add last example --- docs/apis/batch/index.md | 9 + docs/apis/java8.md | 2 +- docs/apis/scala_api_extensions.md | 388 ++++++++++++++++++ docs/apis/streaming/index.md | 10 + .../OnCoGroupDataSet.scala | 41 ++ .../OnCrossDataSet.scala | 38 ++ .../acceptPartialFunctions/OnDataSet.scala | 104 +++++ .../OnGroupedDataSet.scala | 75 ++++ .../OnJoinDataSet.scala | 38 ++ .../flink/api/scala/extensions/package.scala | 201 +++++++++ .../OnConnectedStream.scala | 68 +++ .../acceptPartialFunctions/OnDataStream.scala | 66 +++ .../OnJoinedStream.scala | 38 ++ .../OnKeyedStream.scala | 44 ++ .../OnWindowedStream.scala | 78 ++++ .../api/scala/extensions/package.scala | 202 +++++++++ .../extensions/AcceptPFCoGroupITCase.scala | 230 +++++++++++ .../extensions/AcceptPFFilterITCase.scala | 134 ++++++ .../extensions/AcceptPFFlatMapITCase.scala | 135 ++++++ .../scala/extensions/AcceptPFMapITCase.scala | 174 ++++++++ 20 files changed, 2074 insertions(+), 1 deletion(-) create mode 100644 docs/apis/scala_api_extensions.md create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala create mode 100644 flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala diff --git a/docs/apis/batch/index.md b/docs/apis/batch/index.md index 2a954d9978abe..ff53219173a2a 100644 --- a/docs/apis/batch/index.md +++ b/docs/apis/batch/index.md @@ -729,6 +729,15 @@ val result3 = in.groupBy(0).sortGroup(1, Order.ASCENDING).first(3) +Extraction from tuples, case classes and collections via anonymous pattern matching, like the following: +{% highlight scala %} +val data: DataSet[(Int, String, Double)] = // [...] +data.map { + case (id, name, temperature) => // [...] +} +{% endhighlight %} +is not supported by the API out-of-the-box. To use this feature, you should use a Scala API extension. + diff --git a/docs/apis/java8.md b/docs/apis/java8.md index 53269e3418778..821038bf88e7b 100644 --- a/docs/apis/java8.md +++ b/docs/apis/java8.md @@ -2,7 +2,7 @@ title: "Java 8 Programming Guide" # Top-level navigation top-nav-group: apis -top-nav-pos: 11 +top-nav-pos: 12 top-nav-title: Java 8 --- + +In order to keep a fair amount of consistency between the Scala and Java APIs, some +of the features that allow a high-level of expressiveness in Scala have been left +out from the standard APIs for both batch and streaming. + +If you want to _enjoy the full Scala experience_ you can choose to opt-in to +extensions that enhance the Scala API via implicit conversions. + +To use all the available extensions, you can just add a simple `import` for the +DataSet API + +{% highlight scala %} +import org.apache.flink.api.scala.extensions._ +{% endhighlight %} + +or the DataStream API + +{% highlight scala %} +import org.apache.flink.streaming.api.scala.extensions._ +{% endhighlight %} + +Alternatively, you can import individual extensions _a-là-carte_ to only use those +you prefer. + +## Accept partial functions + +Normally, both the DataSet and DataStream APIs don't accept anonymous pattern +matching functions to deconstruct tuples, case classes or collections, like the +following: + +{% highlight scala %} +val data: DataSet[(Int, String, Double)] = // [...] +data.map { + case (id, name, temperature) => // [...] + // The previous line causes the following compilation error: + // "The argument types of an anonymous function must be fully known. (SLS 8.5)" +} +{% endhighlight %} + +This extension introduces new methods in both the DataSet and DataStream Scala API +that have a one-to-one correspondance in the extended API. These delegating methods +do support anonymous pattern matching functions. + +#### DataSet API + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
MethodOriginalExample
mapWithmap (DataSet) +{% highlight scala %} +data.mapWith { + case (_, value) => value.toString +} +{% endhighlight %} +
mapPartitionWithmapPartition (DataSet) +{% highlight scala %} +data.mapPartitionWith { + case head +: _ => head +} +{% endhighlight %} +
flatMapWithflatMap (DataSet) +{% highlight scala %} +data.flatMapWith { + case (_, name, visitTimes) => visitTimes.map(name -> _) +} +{% endhighlight %} +
filterWithfilter (DataSet) +{% highlight scala %} +data.filterWith { + case Train(_, isOnTime) => isOnTime +} +{% endhighlight %} +
reduceWithreduce (DataSet, GroupedDataSet) +{% highlight scala %} +data.reduceWith { + case ((_, amount1), (_, amount2)) => amount1 + amount2 +} +{% endhighlight %} +
reduceGroupWithreduceGroup (GroupedDataSet) +{% highlight scala %} +data.reduceGroupWith { + case id +: value +: _ => id -> value +} +{% endhighlight %} +
groupingBygroupBy (DataSet) +{% highlight scala %} +data.groupingBy { + case (id, _, _) => id +} +{% endhighlight %} +
sortGroupWithsortGroup (GroupedDataSet) +{% highlight scala %} +grouped.sortGroupWith(Order.ASCENDING) { + case House(_, value) => value +} +{% endhighlight %} +
combineGroupWithcombineGroup (GroupedDataSet) +{% highlight scala %} +grouped.combineGroupWith { + case header +: amounts => amounts.sum +} +{% endhighlight %} +
projectingapply (JoinDataSet, CrossDataSet) +{% highlight scala %} +data1.join(data2).where(0).equalTo(1).projecting { + case ((pk, tx), (products, fk)) => tx -> products +} +{% endhighlight %} +
projectingapply (CoGroupDataSet) +{% highlight scala %} +data1.coGroup(data2).where(0).equalTo(1).projecting { + case (head1 +: _, head2 +: _) => head1 -> head2 +} +{% endhighlight %} +
+ +#### DataStream API + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
MethodOriginalExample
mapWithmap (DataStream) +{% highlight scala %} +data.mapWith { + case (_, value) => value.toString +} +{% endhighlight %} +
mapPartitionWithmapPartition (DataStream) +{% highlight scala %} +data.mapPartitionWith { + case head +: _ => head +} +{% endhighlight %} +
flatMapWithflatMap (DataStream) +{% highlight scala %} +data.flatMapWith { + case (_, name, visits) => visits.map(name -> _) +} +{% endhighlight %} +
filterWithfilter (DataStream) +{% highlight scala %} +data.filterWith { + case Train(_, isOnTime) => isOnTime +} +{% endhighlight %} +
keyingBykeyBy (DataStream) +{% highlight scala %} +data.keyingBy { + case (id, _, _) => id +} +{% endhighlight %} +
mapWithmap (ConnectedDataStream) +{% highlight scala %} +data.mapWith( + map1 = case (_, value) => value.toString, + map2 = case (_, _, value, _) => value + 1 +) +{% endhighlight %} +
flatMapWithflatMap (ConnectedDataStream) +{% highlight scala %} +data.flatMapWith( + flatMap1 = case (_, json) => parse(json), + flatMap2 = case (_, _, json, _) => parse(json) +) +{% endhighlight %} +
keyingBykeyBy (ConnectedDataStream) +{% highlight scala %} +data.keyingBy( + key1 = case (_, timestamp) => timestamp, + key2 = case (id, _, _) => id +) +{% endhighlight %} +
reduceWithreduce (KeyedDataStream, WindowedDataStream) +{% highlight scala %} +data.reduceWith { + case ((_, sum1), (_, sum2) => sum1 + sum2 +} +{% endhighlight %} +
foldWithfold (KeyedDataStream, WindowedDataStream) +{% highlight scala %} +data.foldWith(User(bought = 0)) { + case (User(b), (_, items)) => User(b + items.size) +} +{% endhighlight %} +
applyWithapply (WindowedDataStream) +{% highlight scala %} +data.applyWith(0)( + foldFunction = case (sum, amount) => sum + amount + windowFunction = case (k, w, sum) => // [...] +) +{% endhighlight %} +
projectingapply (JoinedDataStream) +{% highlight scala %} +data1.join(data2).where(0).equalTo(1).projecting { + case ((pk, tx), (products, fk)) => tx -> products +} +{% endhighlight %} +
+ + + +For more information on the semantics of each method, please refer to the +[DataStream](batch/index.html) and [DataSet](streaming/index.html) API documentation. + +To use this extension exclusively, you can add the following `import`: + +{% highlight scala %} +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +{% endhighlight %} + +The following snippet shows a minimal example of how to use these extension methods together: + +{% highlight scala %} +object Main { + import org.apache.flink.api.scala.extensions._ + case class Point(x: Double, y: Double) + def main(args: Array[String]): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + ds.filterWith { + case Point(x, _) => x > 1 + }.reduceWith { + case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + }.mapWith { + case Point(x, y) => (x, y) + }.flatMapWith { + case (x, y) => Seq("x" -> x, "y" -> y) + }.groupingBy { + case (id, value) => id + } + } +} +{% endhighlight %} diff --git a/docs/apis/streaming/index.md b/docs/apis/streaming/index.md index 6d69459248a55..a8998441d14d5 100644 --- a/docs/apis/streaming/index.md +++ b/docs/apis/streaming/index.md @@ -873,6 +873,16 @@ stream.assignTimestamps { timestampExtractor } +Extraction from tuples, case classes and collections via anonymous pattern matching, like the following: +{% highlight scala %} +val data: DataStream[(Int, String, Double)] = // [...] +data.map { + case (id, name, temperature) => // [...] +} +{% endhighlight %} +is not supported by the API out-of-the-box. To use this feature, you should use a Scala API extension. + + diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala new file mode 100644 index 0000000000000..82e579984de2e --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -0,0 +1,41 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{DataSet, CoGroupDataSet} + +import scala.reflect.ClassTag + +class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](dataset: CoGroupDataSet[L, R]) { + + /** + * Co-groups the data sets using the function `fun` to project elements from both in + * the resulting data set + * + * @param fun The function that defines the projection of the co-group operation + * @tparam O The return type of the projection, for which type information must be known + * @return A fully co-grouped data set of Os + */ + def projecting[O: TypeInformation: ClassTag](fun: (Seq[L], Seq[R]) => O): DataSet[O] = + dataset { + (left, right) => + fun(left.to[Seq], right.to[Seq]) + } + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala new file mode 100644 index 0000000000000..4f931f02de7b1 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala @@ -0,0 +1,38 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{DataSet, CrossDataSet} + +import scala.reflect.ClassTag + +class OnCrossDataSet[L: TypeInformation, R: TypeInformation](ds: CrossDataSet[L, R]) { + + /** + * Starting from a cross data set, uses the function `fun` to project elements from + * both the input data sets in the resulting data set + * + * @param fun The function that defines the projection of the join + * @tparam O The return type of the projection, for which type information must be known + * @return A data set of Os + */ + def projecting[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O] = + ds(fun) + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala new file mode 100644 index 0000000000000..2cef0d3f8f735 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala @@ -0,0 +1,104 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{GroupedDataSet, DataSet} + +import scala.reflect.ClassTag + +class OnDataSet[T: TypeInformation](ds: DataSet[T]) { + + /** + * Applies a function `fun` to each item of the data set + * + * @param fun The function to be applied to each item + * @tparam R The type of the items in the returned data set + * @return A dataset of R + */ + def mapWith[R: TypeInformation: ClassTag](fun: T => R): DataSet[R] = + ds.map(fun) + + /** + * Applies a function `fun` to a partition as a whole + * + * @param fun The function to be applied on the whole partition + * @tparam R The type of the items in the returned data set + * @return A dataset of R + */ + def mapPartitionWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + ds.mapPartition { + (it, out) => + out.collect(fun(it.to[Seq])) + } + + /** + * Applies a function `fun` to each item of the dataset, producing a collection of items + * that will be flattened in the resulting data set + * + * @param fun The function to be applied to each item + * @tparam R The type of the items in the returned data set + * @return A dataset of R + */ + def flatMapWith[R: TypeInformation: ClassTag](fun: T => TraversableOnce[R]): DataSet[R] = + ds.flatMap(fun) + + /** + * Applies a predicate `fun` to each item of the data set, keeping only those for which + * the predicate holds + * + * @param fun The predicate to be tested on each item + * @return A dataset of R + */ + def filterWith(fun: T => Boolean): DataSet[T] = + ds.filter(fun) + + /** + * Applies a reducer `fun` to the data set + * + * @param fun The reducing function to be applied on the whole data set + * @tparam R The type of the items in the returned collection + * @return A data set of Rs + */ + def reduceWith[R: TypeInformation](fun: (T, T) => T): DataSet[T] = + ds.reduce(fun) + + /** + * Applies a reducer `fun` to a grouped data set + * + * @param fun The function to be applied to the whole grouping + * @tparam R The type of the items in the returned data set + * @return A dataset of Rs + */ + def reduceGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + ds.reduceGroup { + (it, out) => + out.collect(fun(it.to[Seq])) + } + + /** + * Groups the items according to a grouping function `fun` + * + * @param fun The grouping function + * @tparam K The return type of the grouping function, for which type information must be known + * @return A grouped data set of Ts + */ + def groupingBy[K: TypeInformation](fun: T => K): GroupedDataSet[T] = + ds.groupBy(fun) + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala new file mode 100644 index 0000000000000..1fbc9b5def58d --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala @@ -0,0 +1,75 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.operators.Order +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{DataSet, GroupedDataSet} + +import scala.reflect.ClassTag + +class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { + + /** + * Sorts a group using a sorting function `fun` and an `Order` + * + * @param fun The sorting function, defining the sorting key + * @param order The ordering strategy (ascending, descending, etc.) + * @tparam K The key type + * @return A data set sorted group-wise + */ + def sortGroupWith[K: TypeInformation](order: Order)(fun: T => K): GroupedDataSet[T] = + ds.sortGroup(fun, order) + + /** + * Reduces the whole data set with a reducer `fun` + * + * @param fun The reducing function + * @return A reduced data set of Ts + */ + def reduceWith(fun: (T, T) => T): DataSet[T] = + ds.reduce(fun) + + /** + * Reduces the data set group-wise with a reducer `fun` + * + * @param fun The reducing function + * @tparam R The type of the items in the resulting data set + * @return A data set of Rs reduced group-wise + */ + def reduceGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + ds.reduceGroup { + (it, out) => + out.collect(fun(it.to[Seq])) + } + + /** + * Same as a reducing operation but only acts locally, + * ideal to perform pre-aggregation before a reduction. + * + * @param fun The reducing function + * @tparam R The type of the items in the resulting data set + * @return A data set of Rs reduced group-wise + */ + def combineGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + ds.combineGroup { + (it, out) => + out.collect(fun(it.to[Seq])) + } + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala new file mode 100644 index 0000000000000..a2ae49c959a93 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala @@ -0,0 +1,38 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{DataSet, JoinDataSet} + +import scala.reflect.ClassTag + +class OnJoinDataSet[L: TypeInformation, R: TypeInformation](dataset: JoinDataSet[L, R]) { + + /** + * Joins the data sets using the function `fun` to project elements from both in the + * resulting data set + * + * @param fun The function that defines the projection of the join + * @tparam O The return type of the projection, for which type information must be known + * @return A fully joined data set of Os + */ + def projecting[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O] = + dataset(fun) + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala new file mode 100644 index 0000000000000..81f3743fe0f80 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -0,0 +1,201 @@ +/* + * 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.api.scala + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.extensions.acceptPartialFunctions._ + +import scala.reflect.ClassTag + +package object extensions { + + /** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnDataSet[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = + new OnDataSet[T](ds) + + /** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnJoinDataSet[L: TypeInformation, R: TypeInformation]( + ds: JoinDataSet[L, R]): OnJoinDataSet[L, R] = + new OnJoinDataSet[L, R](ds) + + /** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnCrossDataSet[L: TypeInformation, R: TypeInformation]( + ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = + new OnCrossDataSet[L, R](ds) + + /** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnGroupedDataSet[T: TypeInformation: ClassTag]( + ds: GroupedDataSet[T]): + OnGroupedDataSet[T] = + new OnGroupedDataSet[T](ds) + /** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnCoGroupDataSet[L: TypeInformation, R: TypeInformation]( + ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = + new OnCoGroupDataSet[L, R](ds) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala new file mode 100644 index 0000000000000..cdd0ee2d7f1ee --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala @@ -0,0 +1,68 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.{DataStream, ConnectedStreams} + +class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( + ds: ConnectedStreams[IN1, IN2]) { + + /** + * Applies a CoMap transformation on the connected streams. + * + * The transformation consists of two separate functions, where + * the first one is called for each element of the first connected stream, + * and the second one is called for each element of the second connected stream. + * + * @param map1 Function called per element of the first input. + * @param map2 Function called per element of the second input. + * @return The resulting data stream. + */ + def mapWith[R: TypeInformation](map1: IN1 => R, map2: IN2 => R): DataStream[R] = + ds.map(map1, map2) + + /** + * Applies a CoFlatMap transformation on the connected streams. + * + * The transformation consists of two separate functions, where + * the first one is called for each element of the first connected stream, + * and the second one is called for each element of the second connected stream. + * + * @param flatMap1 Function called per element of the first input. + * @param flatMap2 Function called per element of the second input. + * @return The resulting data stream. + */ + def flatMapWith[R: TypeInformation]( + flatMap1: IN1 => TraversableOnce[R], flatMap2: IN2 => TraversableOnce[R]): DataStream[R] = + ds.flatMap(flatMap1, flatMap2) + + /** + * Keys the two connected streams together. After this operation, all + * elements with the same key from both streams will be sent to the + * same parallel instance of the transformation functions. + * + * @param key1 The first stream's key function + * @param key2 The second stream's key function + * @return The key-grouped connected streams + */ + def keyingBy[K1: TypeInformation, K2: TypeInformation](key1: IN1 => K1, key2: IN2 => K2): + ConnectedStreams[IN1, IN2] = + ds.keyBy(key1, key2) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala new file mode 100644 index 0000000000000..3ba11babacf46 --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala @@ -0,0 +1,66 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.{KeyedStream, DataStream} + +class OnDataStream[T: TypeInformation](stream: DataStream[T]) { + + /** + * Applies a function `fun` to each item of the stream + * + * @param fun The function to be applied to each item + * @tparam R The type of the items in the returned stream + * @return A dataset of R + */ + def mapWith[R: TypeInformation](fun: T => R): DataStream[R] = + stream.map(fun) + + /** + * Applies a function `fun` to each item of the stream, producing a collection of items + * that will be flattened in the resulting stream + * + * @param fun The function to be applied to each item + * @tparam R The type of the items in the returned stream + * @return A dataset of R + */ + def flatMapWith[R: TypeInformation](fun: T => TraversableOnce[R]): DataStream[R] = + stream.flatMap(fun) + + /** + * Applies a predicate `fun` to each item of the stream, keeping only those for which + * the predicate holds + * + * @param fun The predicate to be tested on each item + * @return A dataset of R + */ + def filterWith(fun: T => Boolean): DataStream[T] = + stream.filter(fun) + + /** + * Keys the items according to a keying function `fun` + * + * @param fun The keying function + * @tparam K The type of the key, for which type information must be known + * @return A stream of Ts keyed by Ks + */ + def keyingBy[K: TypeInformation](fun: T => K): KeyedStream[T, K] = + stream.keyBy(fun) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala new file mode 100644 index 0000000000000..380684810b1c9 --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala @@ -0,0 +1,38 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} +import org.apache.flink.streaming.api.windowing.windows.Window + +class OnJoinedStream[L: TypeInformation, R: TypeInformation, K, W <: Window]( + stream: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) { + + /** + * Completes the join operation with the user function that is executed + * for windowed groups. + * + * @param fun The function that defines the projection of the join + * @tparam O The return type of the projection, for which type information must be known + * @return A fully joined data set of Os + */ + def projecting[O: TypeInformation](fun: (L, R) => O): DataStream[O] = + stream.apply(fun) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala new file mode 100644 index 0000000000000..ceeb8fb5a7add --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala @@ -0,0 +1,44 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} + +class OnKeyedStream[T: TypeInformation, K](stream: KeyedStream[T, K]) { + + /** + * Applies a reducer `fun` to the stream + * + * @param fun The reducing function to be applied on the keyed stream + * @return A data set of Ts + */ + def reduceWith(fun: (T, T) => T): DataStream[T] = + stream.reduce(fun) + + /** + * Folds the stream over a zero element with a reducer `fun` + * + * @param initialValue The zero element + * @param fun The reducing function to be applied on the keyed stream + * @return A data set of Rs + */ + def foldWith[R: TypeInformation](initialValue: R)(fun: (R, T) => R): DataStream[R] = + stream.fold(initialValue)(fun) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala new file mode 100644 index 0000000000000..5e35072cc6d2a --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala @@ -0,0 +1,78 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.{DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.windows.Window + +class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { + + /** + * Applies a reduce function to the window. The window function is called for each evaluation + * of the window for each key individually. The output of the reduce function is interpreted + * as a regular non-windowed stream. + * + * This window will try and pre-aggregate data as much as the window policies permit. + * For example,tumbling time windows can perfectly pre-aggregate the data, meaning that only one + * element per key is stored. Sliding time windows will pre-aggregate on the granularity of the + * slide interval, so a few elements are stored per key (one per slide interval). + * Custom windows may not be able to pre-aggregate, or may need to store extra values in an + * aggregation tree. + * + * @param function The reduce function. + * @return The data stream that is the result of applying the reduce function to the window. + */ + def reduceWith(function: (T, T) => T) = + ds.reduce(function) + + /** + * Applies the given fold function to each window. The window function is called for each + * evaluation of the window for each key individually. The output of the reduce function is + * interpreted as a regular non-windowed stream. + * + * @param function The fold function. + * @return The data stream that is the result of applying the fold function to the window. + */ + def foldWith[R: TypeInformation](initialValue: R)(function: (R, T) => R) = + ds.fold(initialValue)(function) + + /** + * Applies the given window function to each window. The window function is called for each + * evaluation of the window for each key individually. The output of the window function is + * interpreted as a regular non-windowed stream. + * + * Arriving data is incrementally aggregated using the given fold function. + * + * @param initialValue The initial value of the fold + * @param foldFunction The fold function that is used for incremental aggregation + * @param windowFunction The window function. + * @return The data stream that is the result of applying the window function to the window. + */ + def applyWith[R: TypeInformation](initialValue: R) + (foldFunction: (R, T) => R, + windowFunction: (K, W, R) => TraversableOnce[R]): + DataStream[R] = + ds.apply(initialValue, foldFunction, { + (key, window, items, out) => + items.foreach { + windowFunction(key, window, _).foreach(out.collect) + } + }) + +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala new file mode 100644 index 0000000000000..40f43cfb136a0 --- /dev/null +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -0,0 +1,202 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions._ +import org.apache.flink.streaming.api.windowing.windows.Window + +package object extensions { + + /** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnDataStream[T: TypeInformation](ds: DataStream[T]): + OnDataStream[T] = + new OnDataStream[T](ds) + + /** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnKeyedStream[T: TypeInformation, K](ds: KeyedStream[T, K]): + OnKeyedStream[T, K] = + new OnKeyedStream[T, K](ds) + + /** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnJoinedStream + [L: TypeInformation, R: TypeInformation, K, W <: Window]( + ds: JoinedStreams.WithWindow[L, R, K, W]) = + new OnJoinedStream[L, R, K, W](ds) + + /** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( + ds: ConnectedStreams[IN1, IN2]) = + new OnConnectedStream[IN1, IN2](ds) + + /** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ + implicit def acceptPartialFunctionsOnWindowedStream[T, K, W <: Window]( + ds: WindowedStream[T, K, W]) = + new OnWindowedStream[T, K, W](ds) + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala new file mode 100644 index 0000000000000..83f2e41cf9476 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala @@ -0,0 +1,230 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.scala.util.CollectionDataSets.CustomType +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.util.Collector +import org.junit._ +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +@RunWith(classOf[Parameterized]) +class AcceptPFCoGroupITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + val _tempFolder = new TemporaryFolder() + var resultPath: String = _ + var expectedResult: String = _ + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) + } + + @Test + def testCoGroupOnTuplesWithKeyFieldSelector(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).projecting { + (first, second) => + var sum = 0 + var id = 0 + for (t <- first) { + sum += t._3 + id = t._1 + } + for (t <- second) { + sum += t._3 + id = t._1 + } + (id, sum) + } + coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "1,0\n" + "2,6\n" + "3,24\n" + "4,60\n" + "5,120\n" + } + + @Test + def testCoGroupOnTwoCustomInputsWithKeyExtractors(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val ds2 = CollectionDataSets.getCustomTypeDataSet(env) + + val coGroupDs = ds.coGroup(ds2).where(_.myInt).equalTo(_.myInt) projecting { + (first, second) => + val o = new CustomType(0, 0, "test") + for (c <- first) { + o.myInt = c.myInt + o.myLong += c.myLong + } + for (c <- second) { + o.myInt = c.myInt + o.myLong += c.myLong + } + o + } + coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "1,0,test\n" + "2,6,test\n" + "3,24,test\n" + "4,60,test\n" + "5,120,test\n" + + "6," + "210,test\n" + } + + @Test + def testCorrectnessIfCoGroupReturnsRightInputObjects(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val coGroupDs = ds.coGroup(ds2).where(0).equalTo(0) { + (first, second, out: Collector[(Int, Long, Int, String, Long)]) => + for (t <- second) { + if (t._1 < 4) { + out.collect(t) + } + } + } + coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,3,2,Hallo Welt wie,1\n" + + "3,4,3," + "Hallo Welt wie gehts?,2\n" + "3,5,4,ABC,2\n" + "3,6,5,BCD,3\n" + } + + @Test + def testCoGroupOnTupleWithKeyFieldSelectorAndCustomTypeWithKeyExtractor(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val ds2 = CollectionDataSets.getCustomTypeDataSet(env) + val coGroupDs = ds.coGroup(ds2).where(2).equalTo(_.myInt) projecting { + (first, second) => + var sum = 0L + var id = 0 + for (t <- first) { + sum += t._1 + id = t._3 + } + for (t <- second) { + sum += t.myLong + id = t.myInt + } + (id, sum, "test") + } + coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "0,1,test\n" + "1,2,test\n" + "2,5,test\n" + "3,15,test\n" + "4,33,test\n" + + "5," + "63,test\n" + "6,109,test\n" + "7,4,test\n" + "8,4,test\n" + "9,4,test\n" + "10,5," + + "test\n" + "11,5,test\n" + "12,5,test\n" + "13,5,test\n" + "14,5,test\n" + } + + @Test + def testCoGroupOnCustomTypeWithKeyExtractorAndTupleInputKeyFieldSelector(): Unit = { + /* + * CoGroup on a tuple input with key field selector and a custom type input with + * key extractor + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val ds2 = CollectionDataSets.getCustomTypeDataSet(env) + val coGroupDs = ds2.coGroup(ds).where(_.myInt).equalTo(2).projecting { + (first, second) => + var sum = 0L + var id = 0 + for (t <- first) { + sum += t.myLong + id = t.myInt + } + for (t <- second) { + sum += t._1 + id = t._3 + } + + new CustomType(id, sum, "test") + } + coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "0,1,test\n" + "1,2,test\n" + "2,5,test\n" + "3,15,test\n" + "4,33,test\n" + + "5," + "63,test\n" + "6,109,test\n" + "7,4,test\n" + "8,4,test\n" + "9,4,test\n" + "10,5," + + "test\n" + "11,5,test\n" + "12,5,test\n" + "13,5,test\n" + "14,5,test\n" + } + + @Test + def testCoGroupWithMultipleKeyExtractors(): Unit = { + /* + * CoGroup with multiple key extractors + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets + .get5TupleDataSet(env) + val ds2 = CollectionDataSets.get3TupleDataSet(env) + val coGrouped = ds1.coGroup(ds2).where(t => (t._1, t._5)).equalTo(t => (t._1, t._2)) + .apply { + (first, second, out: Collector[(Int, Long, String)]) => + val strs = first map(_._4) + for (t <- second) { + for (s <- strs) { + out.collect((t._1, t._2, s)) + } + } + } + + coGrouped.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "1,1,Hallo\n" + "2,2,Hallo Welt\n" + "3,2,Hallo Welt wie gehts?\n" + "3,2," + + "ABC\n" + "5,3,HIJ\n" + "5,3,IJK\n" + + } + + @Test + def testCoGroupOnTwoCustomTypesUsingExpressionKeys(): Unit = { + /* + * CoGroup on two custom type inputs using expression keys + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val ds2 = CollectionDataSets.getCustomTypeDataSet(env) + val coGroupDs = ds.coGroup(ds2).where("myInt").equalTo("myInt").projecting { + (first, second) => + val o = new CustomType(0, 0, "test") + for (t <- first) { + o.myInt = t.myInt + o.myLong += t.myLong + } + for (t <- second) { + o.myInt = t.myInt + o.myLong += t.myLong + } + o + } + coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expectedResult = "1,0,test\n" + "2,6,test\n" + "3,24,test\n" + "4,60,test\n" + "5,120,test\n" + + "6," + "210,test\n" + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala new file mode 100644 index 0000000000000..b1d8596ef4848 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala @@ -0,0 +1,134 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit._ +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +@RunWith(classOf[Parameterized]) +class AcceptPFFilterITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testAllRejectingFilter(): Unit = { + /* + * Test all-rejecting filter. + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val filterDs = ds.filterWith( t => false ) + filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "\n" + } + + @Test + def testAllPassingFilter(): Unit = { + /* + * Test all-passing filter. + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val filterDs = ds.filterWith( t => true ) + filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + + "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + + "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + + "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + + "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + + "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + } + + @Test + def testFilterOnStringTupleField(): Unit = { + /* + * Test filter on String tuple field. + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val filterDs = ds.filterWith( _._3.contains("world") ) + filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" + } + + @Test + def testFilterOnIntegerTupleField(): Unit = { + /* + * Test filter on Integer tuple field. + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val filterDs = ds.filterWith( _._1 % 2 == 0 ) + filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," + + "Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," + + "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n" + } + + @Test + def testFilterBasicType(): Unit = { + /* + * Test filter on basic type + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getStringDataSet(env) + val filterDs = ds.filterWith( _.startsWith("H") ) + filterDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + } + + @Test + def testFilterOnCustomType(): Unit = { + /* + * Test filter on custom type + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val filterDs = ds.filterWith( _.myString.contains("a") ) + filterDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala new file mode 100644 index 0000000000000..4385d2d88076b --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala @@ -0,0 +1,135 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit._ +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +@RunWith(classOf[Parameterized]) +class AcceptPFFlatMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testNonPassingFlatMap(): Unit = { + /* + * Test non-passing flatmap + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getStringDataSet(env) + val nonPassingFlatMapDs = ds.flatMapWith( in => if (in.contains("banana")) Some(in) else None ) + nonPassingFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "\n" + } + + @Test + def testDataDuplicatingFlatMap(): Unit = { + /* + * Test data duplicating flatmap + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getStringDataSet(env) + val duplicatingFlatMapDs = ds.flatMapWith( in => Seq(in, in.toUpperCase) ) + duplicatingFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "Hi\n" + "HI\n" + "Hello\n" + "HELLO\n" + "Hello world\n" + "HELLO WORLD\n" + + "Hello world, how are you?\n" + "HELLO WORLD, HOW ARE YOU?\n" + "I am fine.\n" + "I AM " + + "FINE.\n" + "Luke Skywalker\n" + "LUKE SKYWALKER\n" + "Random comment\n" + "RANDOM " + + "COMMENT\n" + "LOL\n" + "LOL\n" + } + + @Test + def testFlatMapWithVaryingNumberOfEmittedTuples(): Unit = { + /* + * Test flatmap with varying number of emitted tuples + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val varyingTuplesMapDs = ds.flatMapWith { + case in @ (i, l, s) => + val numTuples = i % 3 + (0 until numTuples) map { i => in } + } + varyingTuplesMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "2,2,Hello\n" + "2,2,Hello\n" + "4,3,Hello world, " + + "how are you?\n" + "5,3,I am fine.\n" + "5,3,I am fine.\n" + "7,4,Comment#1\n" + "8,4," + + "Comment#2\n" + "8,4,Comment#2\n" + "10,4,Comment#4\n" + "11,5,Comment#5\n" + "11,5," + + "Comment#5\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "14,5,Comment#8\n" + "16,6," + + "Comment#10\n" + "17,6,Comment#11\n" + "17,6,Comment#11\n" + "19,6,Comment#13\n" + "20," + + "6,Comment#14\n" + "20,6,Comment#14\n" + } + + @Test + def testTypeConversionFlatMapperCustomToTuple(): Unit = { + /* + * Test type conversion flatmapper (Custom -> Tuple) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val typeConversionFlatMapDs = ds.flatMapWith { in => Some((in.myInt, in.myLong, in.myString)) } + typeConversionFlatMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, " + + "how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + "4,6," + + "Comment#1\n" + "4,7,Comment#2\n" + "4,8,Comment#3\n" + "4,9,Comment#4\n" + "5,10," + + "Comment#5\n" + "5,11,Comment#6\n" + "5,12,Comment#7\n" + "5,13,Comment#8\n" + "5,14," + + "Comment#9\n" + "6,15,Comment#10\n" + "6,16,Comment#11\n" + "6,17,Comment#12\n" + "6," + + "18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n" + } + + @Test + def testTypeConversionFlatMapperTupleToBasic(): Unit = { + /* + * Test type conversion flatmapper (Tuple -> Basic) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val typeConversionFlatMapDs = ds.flatMapWith { case (_, _, in) => Some(in) } + typeConversionFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + + ".\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + "Comment#3\n" + + "Comment#4\n" + "Comment#5\n" + "Comment#6\n" + "Comment#7\n" + "Comment#8\n" + + "Comment#9\n" + "Comment#10\n" + "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + + "Comment#14\n" + "Comment#15\n" + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala new file mode 100644 index 0000000000000..bf45002cf7913 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala @@ -0,0 +1,174 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.scala.ExecutionEnvironment +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase} +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.junit._ +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +@RunWith(classOf[Parameterized]) +class AcceptPFMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testIdentityMapperWithBasicType(): Unit = { + /* + * Test identity map with basic type + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getStringDataSet(env) + val identityMapDs = ds.mapWith( t => t) + identityMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + + ".\n" + "Luke Skywalker\n" + "Random comment\n" + "LOL\n" + } + + @Test + def testIdentityMapperWithTuple(): Unit = { + /* + * Test identity map with a tuple + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val identityMapDs = ds.mapWith( t => t ) + identityMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + + "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + + "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + + "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + + "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + + "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + } + + @Test + def testTypeConversionMapperCustomToTuple(): Unit = { + /* + * Test type conversion mapper (Custom -> Tuple) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val typeConversionMapDs = ds.mapWith( c => (c.myInt, c.myLong, c.myString) ) + typeConversionMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, " + + "how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + "4,6," + + "Comment#1\n" + "4,7,Comment#2\n" + "4,8,Comment#3\n" + "4,9,Comment#4\n" + "5,10," + + "Comment#5\n" + "5,11,Comment#6\n" + "5,12,Comment#7\n" + "5,13,Comment#8\n" + "5,14," + + "Comment#9\n" + "6,15,Comment#10\n" + "6,16,Comment#11\n" + "6,17,Comment#12\n" + "6," + + "18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n" + } + + @Test + def testTypeConversionMapperTupleToBasic(): Unit = { + /* + * Test type conversion mapper (Tuple -> Basic) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val typeConversionMapDs = ds.mapWith(_._3) + typeConversionMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + + ".\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + "Comment#3\n" + + "Comment#4\n" + "Comment#5\n" + "Comment#6\n" + "Comment#7\n" + "Comment#8\n" + + "Comment#9\n" + "Comment#10\n" + "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + + "Comment#14\n" + "Comment#15\n" + } + + @Test + def testMapperOnTupleIncrementFieldReorderSecondAndThirdFields(): Unit = { + /* + * Test mapper on tuple - Increment Integer field, reorder second and third fields + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val tupleMapDs = ds.mapWith( t => (t._1 + 1, t._3, t._2) ) + tupleMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "2,Hi,1\n" + "3,Hello,2\n" + "4,Hello world,2\n" + "5,Hello world, how are you?," + + "3\n" + "6,I am fine.,3\n" + "7,Luke Skywalker,3\n" + "8,Comment#1,4\n" + "9,Comment#2," + + "4\n" + "10,Comment#3,4\n" + "11,Comment#4,4\n" + "12,Comment#5,5\n" + "13,Comment#6," + + "5\n" + "14,Comment#7,5\n" + "15,Comment#8,5\n" + "16,Comment#9,5\n" + "17,Comment#10," + + "6\n" + "18,Comment#11,6\n" + "19,Comment#12,6\n" + "20,Comment#13,6\n" + "21," + + "Comment#14,6\n" + "22,Comment#15,6\n" + } + + @Test + def testMapperOnCustomLowercaseString(): Unit = { + /* + * Test mapper on Custom - lowercase myString + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val customMapDs = ds.mapWith { c => c.myString = c.myString.toLowerCase; c } + customMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) + env.execute() + expected = "1,0,hi\n" + "2,1,hello\n" + "2,2,hello world\n" + "3,3,hello world, " + + "how are you?\n" + "3,4,i am fine.\n" + "3,5,luke skywalker\n" + "4,6," + + "comment#1\n" + "4,7,comment#2\n" + "4,8,comment#3\n" + "4,9,comment#4\n" + "5,10," + + "comment#5\n" + "5,11,comment#6\n" + "5,12,comment#7\n" + "5,13,comment#8\n" + "5,14," + + "comment#9\n" + "6,15,comment#10\n" + "6,16,comment#11\n" + "6,17,comment#12\n" + "6," + + "18,comment#13\n" + "6,19,comment#14\n" + "6,20,comment#15\n" + } + + @Test + def testMapperIfUDFReturnsInputObjectIncrementFirstFieldOfTuple(): Unit = { + /* + * Test mapper if UDF returns input object - increment first field of a tuple + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env).mapWith { + t => MutableTuple3(t._1, t._2, t._3) + } + val inputObjMapDs = ds.mapWith { t => t._1 = t._1 + 1; t } + inputObjMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "2,1,Hi\n" + "3,2,Hello\n" + "4,2,Hello world\n" + "5,3,Hello world, " + + "how are you?\n" + "6,3,I am fine.\n" + "7,3,Luke Skywalker\n" + "8,4," + + "Comment#1\n" + "9,4,Comment#2\n" + "10,4,Comment#3\n" + "11,4,Comment#4\n" + "12,5," + + "Comment#5\n" + "13,5,Comment#6\n" + "14,5,Comment#7\n" + "15,5,Comment#8\n" + "16,5," + + "Comment#9\n" + "17,6,Comment#10\n" + "18,6,Comment#11\n" + "19,6,Comment#12\n" + "20," + + "6,Comment#13\n" + "21,6,Comment#14\n" + "22,6,Comment#15\n" + } + +} From 83fa93afcd247c405830674cc89312ea188b5c68 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 5 Mar 2016 01:01:16 +0100 Subject: [PATCH 02/39] [FLINK-1159] Add example for --- docs/apis/scala_api_extensions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/apis/scala_api_extensions.md b/docs/apis/scala_api_extensions.md index e0c1e2f701ba5..4e91a477bbc2b 100644 --- a/docs/apis/scala_api_extensions.md +++ b/docs/apis/scala_api_extensions.md @@ -183,6 +183,10 @@ grouped.combineGroupWith { data1.join(data2).where(0).equalTo(1).projecting { case ((pk, tx), (products, fk)) => tx -> products } + +data1.cross(data2).projecting { + case ((a, _), (_, b) => a -> b +} {% endhighlight %} From 40fbc048c7bffe6ffbe1d7c4668de3af2c9adcbe Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 5 Mar 2016 01:01:56 +0100 Subject: [PATCH 03/39] [FLINK-1159] Add tests for Cross, Reduce and Join --- .../extensions/AcceptPFCrossITCase.scala | 153 ++++++++++++++++ .../extensions/AcceptPFReduceITCase.scala | 144 ++++++++++++++++ .../scala/extensions/AccetPFJoinITCase.scala | 163 ++++++++++++++++++ 3 files changed, 460 insertions(+) create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala create mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala new file mode 100644 index 0000000000000..179a13c821776 --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala @@ -0,0 +1,153 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.common.functions.RichCrossFunction +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.scala.util.CollectionDataSets.CustomType +import org.apache.flink.configuration.Configuration +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +import scala.collection.JavaConverters._ + +@RunWith(classOf[Parameterized]) +class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testCorrectnessOfCrossOnTwoTupleInputs(): Unit = { + /* + * check correctness of cross on two tuple inputs + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getSmall5TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) + val crossDs = ds.cross(ds2).projecting { + case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + } + crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + + expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + + "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + + "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + + "Hallo Welt wieHallo Welt wie\n" + } + + @Test + def testCorrectnessOfCrossIfUDFReturnsLeftInput(): Unit = { + /* + * check correctness of cross if UDF returns left input object + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) + val crossDs = ds.cross(ds2).projecting { + case (l @ (_, _, _), _) => l + } + crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + + expected = "1,1,Hi\n" + "1,1,Hi\n" + "1,1,Hi\n" + "2,2,Hello\n" + "2,2,Hello\n" + "2,2," + + "Hello\n" + "3,2,Hello world\n" + "3,2,Hello world\n" + "3,2,Hello world\n" + } + + @Test + def testCorrectnessOfCrossIfUDFReturnsRightInput(): Unit = { + /* + * check correctness of cross if UDF returns right input object + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) + val crossDs = ds.cross(ds2).projecting { + case (_, r @ (_, _, _)) => r + } + crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + + expected = "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt," + + "2\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n" + "2,3,2,Hallo Welt wie," + + "1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,3,2,Hallo Welt wie,1\n" + } + + @Test + def testCorrectnessOfCrossWithHuge(): Unit = { + /* + * check correctness of crossWithHuge (only correctness of result -> should be the same + * as with normal cross) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.getSmall5TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) + val crossDs = ds.crossWithHuge(ds2).projecting { + case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + } + crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + + "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + + "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + + "Hallo Welt wieHallo Welt wie\n" + } + + @Test + def testCorrectnessOfCrossWithTiny(): Unit = { + /* + * check correctness of crossWithTiny (only correctness of result -> should be the same + * as with normal cross) + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets + .getSmall5TupleDataSet(env) + val ds2 = CollectionDataSets + .getSmall5TupleDataSet(env) + val crossDs = ds.crossWithTiny(ds2).projecting { + case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + } + crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + + "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + + "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + + "Hallo Welt wieHallo Welt wie\n" + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala new file mode 100644 index 0000000000000..69378255b977a --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala @@ -0,0 +1,144 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.common.functions.RichReduceFunction +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 +import org.apache.flink.configuration.Configuration +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +import scala.collection.JavaConverters._ + +@RunWith(classOf[Parameterized]) +class AcceptPFReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testReduceOnTuplesWithKeyFieldSelector(): Unit = { + /* + * Reduce on tuples with key field selector + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val reduceDs = ds.groupBy(1) + .reduceWith { case ((i1, l1, _), (i2, _, _)) => (i1 + i2, l1, "B-)") } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n" + } + + @Test + def testReduceOnTuplesWithMultipleKeyFieldSelectors(): Unit = { + /* + * Reduce on tuples with multiple key field selectors + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val reduceDs = ds.groupBy(4, 0) + .reduceWith { + case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => (in11, in12 + in22, 0, "P-)", in15) } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + + "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + + "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" + } + + @Test + def testReduceOnTuplesWithKeyExtractor(): Unit = { + /* + * Reduce on tuples with key extractor + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val reduceDs = ds.groupBy(_._2) + .reduceWith { case ((i1, l1, _), (i2, _, _)) => (i1 + i2, l1, "B-)") } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n" + } + + @Test + def testAllReduceForTuple(): Unit = { + /* + * All-reduce for tuple + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get3TupleDataSet(env) + val reduceDs = + ds.reduceWith { case ((i1, l1, _), (i2, l2, _)) => (i1 + i2, l1 + l2, "Hello World") } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "231,91,Hello World\n" + } + + @Test + def testReduceWithATupleReturningKeySelector(): Unit = { + /* + * Reduce with a Tuple-returning KeySelector + */ + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val reduceDs = ds.groupBy(t => (t._1, t._5)) + .reduceWith { case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => + (in11, in12 + in22, 0, "P-)", in15) } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + + "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + + "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" + } + + @Test + def testReduceOnGroupedDSByExpressionKey(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val ds = CollectionDataSets.get5TupleDataSet(env) + val reduceDs = ds.groupBy("_5", "_1") + .reduceWith { case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => + (in11, in12 + in22, 0, "P-)", in15) } + reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + + "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + + "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala new file mode 100644 index 0000000000000..36a59e114a5ba --- /dev/null +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala @@ -0,0 +1,163 @@ +/* + * 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.api.scala.extensions + +import org.apache.flink.api.common.functions.RichJoinFunction +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.configuration.Configuration +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{After, Before, Rule, Test} + +import scala.collection.JavaConverters._ + +@RunWith(classOf[Parameterized]) +class AccetPFJoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { + private var resultPath: String = null + private var expected: String = null + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + @Before + def before(): Unit = { + resultPath = tempFolder.newFile().toURI.toString + } + + @After + def after(): Unit = { + TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) + } + + @Test + def testUDFJoinOnTuplesWithKeyFieldPositions(): Unit = { + /* + * UDF Join on tuples with key field positions + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { + case ((_, _, l), (_, _, _, r, _)) => l -> r + } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" + } + + @Test + def testUDFJoinOnTuplesWithMultipleKeyFieldPositions(): Unit = { + /* + * UDF Join on tuples with multiple key field positions + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.get3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.join(ds2).where(0, 1).equalTo(0, 4).projecting { + case ((_, _, l), (_, _, _, r, _)) => l -> r + } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + + "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n" + } + + @Test + def testJoinWithHuge(): Unit = { + /* + * Join with Huge + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.joinWithHuge(ds2).where(1).equalTo(1).projecting { + case ((_, _, l), (_, _, _, r, _)) => l -> r + } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" + } + + @Test + def testJoinWithTiny(): Unit = { + /* + * Join with Tiny + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.joinWithTiny(ds2).where(1).equalTo(1).projecting { + case ((_, _, l), (_, _, _, r, _)) => l -> r + } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" + } + + @Test + def testJoinThatReturnsTheLeftInputObject(): Unit = { + /* + * Join that returns the left input object + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (l @ (_, _, _), _) => l } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + } + + @Test + def testJoinThatReturnsTheRightInputObject(): Unit = { + /* + * Join that returns the right input object + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (_, r @ (_, _, _)) => r } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n" + } + + @Test + def testUDFJoinOnTuplesWithTupleReturningKeySelectors(): Unit = { + /* + * UDF Join on tuples with tuple-returning key selectors + */ + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val ds1 = CollectionDataSets.get3TupleDataSet(env) + val ds2 = CollectionDataSets.get5TupleDataSet(env) + val joinDs = ds1.join(ds2).where( t => (t._1, t._2)).equalTo( t => (t._1, t._5)).projecting { + case ((_, _, l), (_, _, _, r, _)) => l -> r + } + joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) + env.execute() + expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + + "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n" + } + +} From 7c2728f1d6d80f1236b0a2ab4f3a987ce3c9dad4 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 5 Mar 2016 11:37:08 +0100 Subject: [PATCH 04/39] [FLINK-1159] Fix test compile errors --- .../extensions/AcceptPFCrossITCase.scala | 32 +++++++++---------- .../scala/extensions/AccetPFJoinITCase.scala | 2 +- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala index 179a13c821776..3af53b14db2ec 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala @@ -57,10 +57,10 @@ class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestB * check correctness of cross on two tuple inputs */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getSmall5TupleDataSet(env) + val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds.cross(ds2).projecting { - case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + val crossDs = ds1.cross(ds2).projecting { + case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) } crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) env.execute() @@ -77,9 +77,9 @@ class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestB * check correctness of cross if UDF returns left input object */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getSmall3TupleDataSet(env) + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds.cross(ds2).projecting { + val crossDs = ds1.cross(ds2).projecting { case (l @ (_, _, _), _) => l } crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) @@ -95,10 +95,10 @@ class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestB * check correctness of cross if UDF returns right input object */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getSmall3TupleDataSet(env) + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds.cross(ds2).projecting { - case (_, r @ (_, _, _)) => r + val crossDs = ds1.cross(ds2).projecting { + case (_, r @ (_, _, _, _, _)) => r } crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) env.execute() @@ -115,10 +115,10 @@ class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestB * as with normal cross) */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getSmall5TupleDataSet(env) + val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds.crossWithHuge(ds2).projecting { - case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + val crossDs = ds1.crossWithHuge(ds2).projecting { + case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) } crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) env.execute() @@ -135,12 +135,10 @@ class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestB * as with normal cross) */ val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets - .getSmall5TupleDataSet(env) - val ds2 = CollectionDataSets - .getSmall5TupleDataSet(env) - val crossDs = ds.crossWithTiny(ds2).projecting { - case ((_, _, l3, l4), (_, _, r3, r4)) => (l3 + r3, l4 + r4) + val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) + val crossDs = ds1.crossWithTiny(ds2).projecting { + case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) } crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) env.execute() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala index 36a59e114a5ba..212e69db6d939 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala @@ -137,7 +137,7 @@ class AccetPFJoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (_, r @ (_, _, _)) => r } + val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (_, r @ (_, _, _, _, _)) => r } joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) env.execute() expected = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n" From 55eff65483ea95ae13e1285eef74289a146c9eb2 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 5 Mar 2016 11:42:53 +0100 Subject: [PATCH 05/39] [FLINK-1159] Fix JonedStreams compile errors --- .../apache/flink/streaming/api/scala/extensions/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index 40f43cfb136a0..eca79ea946f23 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -126,7 +126,7 @@ package object extensions { */ implicit def acceptPartialFunctionsOnJoinedStream [L: TypeInformation, R: TypeInformation, K, W <: Window]( - ds: JoinedStreams.WithWindow[L, R, K, W]) = + ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) = new OnJoinedStream[L, R, K, W](ds) /** From 4433e1236e731b4a003d5a4a235fc37c465b31ac Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 5 Mar 2016 11:43:11 +0100 Subject: [PATCH 06/39] [FLINK-1159] Remove unused imports --- .../apache/flink/streaming/api/scala/ConnectedStreams.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala index 669f12e58c117..141625ed8b8e5 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala @@ -18,14 +18,13 @@ package org.apache.flink.streaming.api.scala -import org.apache.flink.annotation.{PublicEvolving, Internal, Public} +import org.apache.flink.annotation.{Internal, Public, PublicEvolving} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.functions.KeySelector import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream, SingleOutputStreamOperator, KeyedStream} +import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream} import org.apache.flink.streaming.api.functions.co.{CoFlatMapFunction, CoMapFunction} import org.apache.flink.streaming.api.operators.TwoInputStreamOperator -import org.apache.flink.streaming.api.transformations.TwoInputTransformation import org.apache.flink.util.Collector /** From 920885a31f7444fe8e3f7baea5c192ace3d0a51e Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sat, 12 Mar 2016 22:54:31 +0100 Subject: [PATCH 07/39] [FLINK-1159] Testing transformations without running --- .../scala/extensions/AcceptPFMapITCase.scala | 155 ++++++------------ 1 file changed, 47 insertions(+), 108 deletions(-) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala index bf45002cf7913..bfdfe6332637b 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala @@ -17,158 +17,97 @@ */ package org.apache.flink.api.scala.extensions -import org.apache.flink.api.scala.ExecutionEnvironment -import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 +import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} +import org.apache.flink.configuration.Configuration import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.optimizer.Optimizer +import org.apache.flink.optimizer.dag.{MapNode, OptimizerNode} +import org.apache.flink.optimizer.plan.{PlanNode, SinkPlanNode, SourcePlanNode} +import org.apache.flink.util.TestLogger +import org.junit.Assert.assertTrue import org.junit._ -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -@RunWith(classOf[Parameterized]) -class AcceptPFMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() +import scala.collection.JavaConversions._ - @Rule - def tempFolder = _tempFolder +class AcceptPFMapITCase extends TestLogger { - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } + private val optimizer = new Optimizer(new Configuration) + private val unusedResultPath = "UNUSED" - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } + def isTransformation(n: PlanNode): Boolean = + !n.isInstanceOf[SourcePlanNode] && !n.isInstanceOf[SinkPlanNode] + + private def getOptimizerTransformationNodes(env: ExecutionEnvironment): Iterable[OptimizerNode] = + for { + node <- optimizer.compile(env.createProgramPlan("UNUSED")).getAllNodes + transformation = node.getOptimizerNode if isTransformation(node) + } yield transformation @Test def testIdentityMapperWithBasicType(): Unit = { - /* - * Test identity map with basic type - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getStringDataSet(env) - val identityMapDs = ds.mapWith( t => t) - identityMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + - ".\n" + "Luke Skywalker\n" + "Random comment\n" + "LOL\n" + val identityMapDs = ds.mapWith(identity) + identityMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } @Test def testIdentityMapperWithTuple(): Unit = { - /* - * Test identity map with a tuple - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - val identityMapDs = ds.mapWith( t => t ) - identityMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + - "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + - "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + - "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + - "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + - "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val identityMapDs = ds.mapWith(identity) + identityMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } @Test def testTypeConversionMapperCustomToTuple(): Unit = { - /* - * Test type conversion mapper (Custom -> Tuple) - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) val typeConversionMapDs = ds.mapWith( c => (c.myInt, c.myLong, c.myString) ) - typeConversionMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, " + - "how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + "4,6," + - "Comment#1\n" + "4,7,Comment#2\n" + "4,8,Comment#3\n" + "4,9,Comment#4\n" + "5,10," + - "Comment#5\n" + "5,11,Comment#6\n" + "5,12,Comment#7\n" + "5,13,Comment#8\n" + "5,14," + - "Comment#9\n" + "6,15,Comment#10\n" + "6,16,Comment#11\n" + "6,17,Comment#12\n" + "6," + - "18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n" + typeConversionMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } @Test def testTypeConversionMapperTupleToBasic(): Unit = { - /* - * Test type conversion mapper (Tuple -> Basic) - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - val typeConversionMapDs = ds.mapWith(_._3) - typeConversionMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + - ".\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + "Comment#3\n" + - "Comment#4\n" + "Comment#5\n" + "Comment#6\n" + "Comment#7\n" + "Comment#8\n" + - "Comment#9\n" + "Comment#10\n" + "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + - "Comment#14\n" + "Comment#15\n" + val typeConversionMapDs = ds.mapWith { case (_, _, a) => a } + typeConversionMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } @Test def testMapperOnTupleIncrementFieldReorderSecondAndThirdFields(): Unit = { - /* - * Test mapper on tuple - Increment Integer field, reorder second and third fields - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - val tupleMapDs = ds.mapWith( t => (t._1 + 1, t._3, t._2) ) - tupleMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "2,Hi,1\n" + "3,Hello,2\n" + "4,Hello world,2\n" + "5,Hello world, how are you?," + - "3\n" + "6,I am fine.,3\n" + "7,Luke Skywalker,3\n" + "8,Comment#1,4\n" + "9,Comment#2," + - "4\n" + "10,Comment#3,4\n" + "11,Comment#4,4\n" + "12,Comment#5,5\n" + "13,Comment#6," + - "5\n" + "14,Comment#7,5\n" + "15,Comment#8,5\n" + "16,Comment#9,5\n" + "17,Comment#10," + - "6\n" + "18,Comment#11,6\n" + "19,Comment#12,6\n" + "20,Comment#13,6\n" + "21," + - "Comment#14,6\n" + "22,Comment#15,6\n" + val tupleMapDs = ds.mapWith { case (a, b, c) => (a + 1, c, b) } + tupleMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } @Test def testMapperOnCustomLowercaseString(): Unit = { - /* - * Test mapper on Custom - lowercase myString - */ val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.getCustomTypeDataSet(env) val customMapDs = ds.mapWith { c => c.myString = c.myString.toLowerCase; c } - customMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "1,0,hi\n" + "2,1,hello\n" + "2,2,hello world\n" + "3,3,hello world, " + - "how are you?\n" + "3,4,i am fine.\n" + "3,5,luke skywalker\n" + "4,6," + - "comment#1\n" + "4,7,comment#2\n" + "4,8,comment#3\n" + "4,9,comment#4\n" + "5,10," + - "comment#5\n" + "5,11,comment#6\n" + "5,12,comment#7\n" + "5,13,comment#8\n" + "5,14," + - "comment#9\n" + "6,15,comment#10\n" + "6,16,comment#11\n" + "6,17,comment#12\n" + "6," + - "18,comment#13\n" + "6,19,comment#14\n" + "6,20,comment#15\n" - } - - @Test - def testMapperIfUDFReturnsInputObjectIncrementFirstFieldOfTuple(): Unit = { - /* - * Test mapper if UDF returns input object - increment first field of a tuple - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env).mapWith { - t => MutableTuple3(t._1, t._2, t._3) - } - val inputObjMapDs = ds.mapWith { t => t._1 = t._1 + 1; t } - inputObjMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "2,1,Hi\n" + "3,2,Hello\n" + "4,2,Hello world\n" + "5,3,Hello world, " + - "how are you?\n" + "6,3,I am fine.\n" + "7,3,Luke Skywalker\n" + "8,4," + - "Comment#1\n" + "9,4,Comment#2\n" + "10,4,Comment#3\n" + "11,4,Comment#4\n" + "12,5," + - "Comment#5\n" + "13,5,Comment#6\n" + "14,5,Comment#7\n" + "15,5,Comment#8\n" + "16,5," + - "Comment#9\n" + "17,6,Comment#10\n" + "18,6,Comment#11\n" + "19,6,Comment#12\n" + "20," + - "6,Comment#13\n" + "21,6,Comment#14\n" + "22,6,Comment#15\n" + customMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) + val nodes = getOptimizerTransformationNodes(env) + assertTrue("The plan should contain 1 transformation", nodes.size == 1) + assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) } } From 8f208f393f2a0463d8c989303e819651077c02b1 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 13:12:29 +0100 Subject: [PATCH 08/39] [FLINK-1159] Using Streams instead os Seqs --- .../acceptPartialFunctions/OnCoGroupDataSet.scala | 4 ++-- .../extensions/acceptPartialFunctions/OnDataSet.scala | 8 ++++---- .../acceptPartialFunctions/OnGroupedDataSet.scala | 8 ++++---- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala index 82e579984de2e..d0ffef8696214 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -32,10 +32,10 @@ class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](dataset: CoGroupD * @tparam O The return type of the projection, for which type information must be known * @return A fully co-grouped data set of Os */ - def projecting[O: TypeInformation: ClassTag](fun: (Seq[L], Seq[R]) => O): DataSet[O] = + def projecting[O: TypeInformation: ClassTag](fun: (Stream[L], Stream[R]) => O): DataSet[O] = dataset { (left, right) => - fun(left.to[Seq], right.to[Seq]) + fun(left.toStream, right.toStream) } } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala index 2cef0d3f8f735..c9b73117141a3 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala @@ -41,10 +41,10 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of R */ - def mapPartitionWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + def mapPartitionWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.mapPartition { (it, out) => - out.collect(fun(it.to[Seq])) + out.collect(fun(it.toStream)) } /** @@ -85,10 +85,10 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of Rs */ - def reduceGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + def reduceGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.reduceGroup { (it, out) => - out.collect(fun(it.to[Seq])) + out.collect(fun(it.toStream)) } /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala index 1fbc9b5def58d..4bbe192823920 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala @@ -52,10 +52,10 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @tparam R The type of the items in the resulting data set * @return A data set of Rs reduced group-wise */ - def reduceGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + def reduceGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.reduceGroup { (it, out) => - out.collect(fun(it.to[Seq])) + out.collect(fun(it.toStream)) } /** @@ -66,10 +66,10 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @tparam R The type of the items in the resulting data set * @return A data set of Rs reduced group-wise */ - def combineGroupWith[R: TypeInformation: ClassTag](fun: Seq[T] => R): DataSet[R] = + def combineGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.combineGroup { (it, out) => - out.collect(fun(it.to[Seq])) + out.collect(fun(it.toStream)) } } From 565d005204bf6ac09c097d4de062cd17f09702b9 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 13:17:05 +0100 Subject: [PATCH 09/39] [FLINK-1159] Window function on a stream instead of individual items --- .../acceptPartialFunctions/OnWindowedStream.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala index 5e35072cc6d2a..859c0be994f27 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala @@ -66,13 +66,11 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { */ def applyWith[R: TypeInformation](initialValue: R) (foldFunction: (R, T) => R, - windowFunction: (K, W, R) => TraversableOnce[R]): + windowFunction: (K, W, Stream[R]) => TraversableOnce[R]): DataStream[R] = ds.apply(initialValue, foldFunction, { (key, window, items, out) => - items.foreach { - windowFunction(key, window, _).foreach(out.collect) - } + windowFunction(key, window, items.toStream).foreach(out.collect) }) } From 0a17676385c7102f47e6685ad13b6ea4d3ad46b9 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 13:22:47 +0100 Subject: [PATCH 10/39] [FLINK-1159] Reformat long function signature --- .../acceptPartialFunctions/OnWindowedStream.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala index 859c0be994f27..417b9ac6dda15 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala @@ -64,10 +64,11 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { * @param windowFunction The window function. * @return The data stream that is the result of applying the window function to the window. */ - def applyWith[R: TypeInformation](initialValue: R) - (foldFunction: (R, T) => R, - windowFunction: (K, W, Stream[R]) => TraversableOnce[R]): - DataStream[R] = + def applyWith[R: TypeInformation]( + initialValue: R)( + foldFunction: (R, T) => R, + windowFunction: (K, W, Stream[R]) => TraversableOnce[R]) + : DataStream[R] = ds.apply(initialValue, foldFunction, { (key, window, items, out) => windowFunction(key, window, items.toStream).foreach(out.collect) From 03dd7b4c4edb65a5d0b6acff56a5ab72ed824c08 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 15:46:25 +0100 Subject: [PATCH 11/39] [FLINK-1159] Add unit tests for OnDataSet --- .../acceptPartialFunctions/KeyValuePair.scala | 8 + .../OnDataSetTest.scala | 176 ++++++++++++++++++ 2 files changed, 184 insertions(+) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala new file mode 100644 index 0000000000000..b20b0d2df6efa --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala @@ -0,0 +1,8 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +/** + * Simple case class to test the `acceptPartialFunctions` extension + * @param id A numerical identifier + * @param value A textual value + */ +private[acceptPartialFunctions] case class KeyValuePair(id: Int, value: String) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala new file mode 100644 index 0000000000000..e1fb4f6a4d2af --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala @@ -0,0 +1,176 @@ +/* + * 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.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.java.operators._ +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnDataSet +import org.apache.flink.util.TestLogger +import org.junit.Test +import org.scalatest.junit.JUnitSuiteLike + +class OnDataSetTest extends TestLogger with JUnitSuiteLike { + + private val env = ExecutionEnvironment.getExecutionEnvironment + private val tuples = env.fromElements(1 -> "hello", 2 -> "world") + private val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + + @Test + def testMapWithOnTuple(): Unit = { + val test = + tuples.mapWith { + case (id, value) => s"$id $value" + } + assert(test.javaSet.isInstanceOf[MapOperator[_, _]], + "mapWith should produce a MapOperator") + } + + @Test + def testMapWithOnCaseClass(): Unit = { + val test = + caseObjects.mapWith { + case KeyValuePair(id, value) => s"$id $value" + } + assert(test.javaSet.isInstanceOf[MapOperator[_, _]], + "mapWith should produce a MapOperator") + } + + @Test + def testMapPartitionWithOnTuple(): Unit = { + val test = + tuples.mapPartitionWith { + case (id, value) #:: _ => s"$id $value" + } + assert(test.javaSet.isInstanceOf[MapPartitionOperator[_, _]], + "mapPartitionWith should produce a MapPartitionOperator") + } + + @Test + def testMapPartitionWithOnCaseClass(): Unit = { + val test = + caseObjects.mapPartitionWith { + case KeyValuePair(id, value) #:: _ => s"$id $value" + } + assert(test.javaSet.isInstanceOf[MapPartitionOperator[_, _]], + "mapPartitionWith should produce a MapPartitionOperator") + } + + @Test + def testFlatMapWithOnTuple(): Unit = { + val test = + tuples.flatMapWith { + case (id, value) => List(id.toString, value) + } + assert(test.javaSet.isInstanceOf[FlatMapOperator[_, _]], + "flatMapWith should produce a FlatMapOperator") + } + + @Test + def testFlatMapWithOnCaseClass(): Unit = { + val test = + caseObjects.flatMapWith { + case KeyValuePair(id, value) => List(id.toString, value) + } + assert(test.javaSet.isInstanceOf[FlatMapOperator[_, _]], + "flatMapWith should produce a FlatMapOperator") + } + + @Test + def testFilterWithOnTuple(): Unit = { + val test = + tuples.filterWith { + case (id, value) => id == 1 + } + assert(test.javaSet.isInstanceOf[FilterOperator[_]], + "filterWith should produce a FilterOperator") + } + + @Test + def testFilterWithOnCaseClass(): Unit = { + val test = + caseObjects.filterWith { + case KeyValuePair(id, value) => id == 1 + } + assert(test.javaSet.isInstanceOf[FilterOperator[_]], + "filterWith should produce a FilterOperator") + } + + @Test + def testReduceWithOnTuple(): Unit = { + val test = + tuples.reduceWith { + case ((_, v1), (_, v2)) => (0, s"$v1 $v2") + } + assert(test.javaSet.isInstanceOf[ReduceOperator[_]], + "reduceWith should produce a ReduceOperator") + } + + @Test + def testReduceWithOnCaseClass(): Unit = { + val test = + caseObjects.reduceWith { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => KeyValuePair(0, s"$v1 $v2") + } + assert(test.javaSet.isInstanceOf[ReduceOperator[_]], + "reduceWith should produce a ReduceOperator") + } + + @Test + def testReduceGroupWithOnTuple(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + tuples.reduceGroupWith { + case (_, value) #:: _ => accumulator.append(value).append('\n') + } + assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], + "reduceGroupWith should produce a GroupReduceOperator") + } + + @Test + def testReduceGroupWithOnCaseClass(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + caseObjects.reduceGroupWith { + case KeyValuePair(_, value) #:: _ => accumulator.append(value).append('\n') + } + assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], + "reduceGroupWith should produce a GroupReduceOperator") + } + + @Test + def testGroupingByOnTuple(): Unit = { + val test = + tuples.groupingBy { + case (id, _) => id + } + assert(test.isInstanceOf[GroupedDataSet[_]], + "groupingBy should produce a GroupedDataSet") + } + + @Test + def testGroupingByOnCaseClass(): Unit = { + val test = + caseObjects.groupingBy { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[GroupedDataSet[_]], + "groupingBy should produce a GroupedDataSet") + } + +} From f253c16f197839fecb2bd9f3b9d3ed8e6454933d Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 15:46:54 +0100 Subject: [PATCH 12/39] [FLINK-1159] Remove useless integration tests --- .../extensions/AcceptPFCoGroupITCase.scala | 230 ------------------ .../extensions/AcceptPFCrossITCase.scala | 151 ------------ .../extensions/AcceptPFFilterITCase.scala | 134 ---------- .../extensions/AcceptPFFlatMapITCase.scala | 135 ---------- .../scala/extensions/AcceptPFMapITCase.scala | 113 --------- .../extensions/AcceptPFReduceITCase.scala | 144 ----------- .../scala/extensions/AccetPFJoinITCase.scala | 163 ------------- 7 files changed, 1070 deletions(-) delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala deleted file mode 100644 index 83f2e41cf9476..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCoGroupITCase.scala +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.util.CollectionDataSets.CustomType -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.apache.flink.util.Collector -import org.junit._ -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -@RunWith(classOf[Parameterized]) -class AcceptPFCoGroupITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - val _tempFolder = new TemporaryFolder() - var resultPath: String = _ - var expectedResult: String = _ - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expectedResult, resultPath) - } - - @Test - def testCoGroupOnTuplesWithKeyFieldSelector(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).projecting { - (first, second) => - var sum = 0 - var id = 0 - for (t <- first) { - sum += t._3 - id = t._1 - } - for (t <- second) { - sum += t._3 - id = t._1 - } - (id, sum) - } - coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "1,0\n" + "2,6\n" + "3,24\n" + "4,60\n" + "5,120\n" - } - - @Test - def testCoGroupOnTwoCustomInputsWithKeyExtractors(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val ds2 = CollectionDataSets.getCustomTypeDataSet(env) - - val coGroupDs = ds.coGroup(ds2).where(_.myInt).equalTo(_.myInt) projecting { - (first, second) => - val o = new CustomType(0, 0, "test") - for (c <- first) { - o.myInt = c.myInt - o.myLong += c.myLong - } - for (c <- second) { - o.myInt = c.myInt - o.myLong += c.myLong - } - o - } - coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "1,0,test\n" + "2,6,test\n" + "3,24,test\n" + "4,60,test\n" + "5,120,test\n" + - "6," + "210,test\n" - } - - @Test - def testCorrectnessIfCoGroupReturnsRightInputObjects(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val coGroupDs = ds.coGroup(ds2).where(0).equalTo(0) { - (first, second, out: Collector[(Int, Long, Int, String, Long)]) => - for (t <- second) { - if (t._1 < 4) { - out.collect(t) - } - } - } - coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,3,2,Hallo Welt wie,1\n" + - "3,4,3," + "Hallo Welt wie gehts?,2\n" + "3,5,4,ABC,2\n" + "3,6,5,BCD,3\n" - } - - @Test - def testCoGroupOnTupleWithKeyFieldSelectorAndCustomTypeWithKeyExtractor(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val ds2 = CollectionDataSets.getCustomTypeDataSet(env) - val coGroupDs = ds.coGroup(ds2).where(2).equalTo(_.myInt) projecting { - (first, second) => - var sum = 0L - var id = 0 - for (t <- first) { - sum += t._1 - id = t._3 - } - for (t <- second) { - sum += t.myLong - id = t.myInt - } - (id, sum, "test") - } - coGroupDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "0,1,test\n" + "1,2,test\n" + "2,5,test\n" + "3,15,test\n" + "4,33,test\n" + - "5," + "63,test\n" + "6,109,test\n" + "7,4,test\n" + "8,4,test\n" + "9,4,test\n" + "10,5," + - "test\n" + "11,5,test\n" + "12,5,test\n" + "13,5,test\n" + "14,5,test\n" - } - - @Test - def testCoGroupOnCustomTypeWithKeyExtractorAndTupleInputKeyFieldSelector(): Unit = { - /* - * CoGroup on a tuple input with key field selector and a custom type input with - * key extractor - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val ds2 = CollectionDataSets.getCustomTypeDataSet(env) - val coGroupDs = ds2.coGroup(ds).where(_.myInt).equalTo(2).projecting { - (first, second) => - var sum = 0L - var id = 0 - for (t <- first) { - sum += t.myLong - id = t.myInt - } - for (t <- second) { - sum += t._1 - id = t._3 - } - - new CustomType(id, sum, "test") - } - coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "0,1,test\n" + "1,2,test\n" + "2,5,test\n" + "3,15,test\n" + "4,33,test\n" + - "5," + "63,test\n" + "6,109,test\n" + "7,4,test\n" + "8,4,test\n" + "9,4,test\n" + "10,5," + - "test\n" + "11,5,test\n" + "12,5,test\n" + "13,5,test\n" + "14,5,test\n" - } - - @Test - def testCoGroupWithMultipleKeyExtractors(): Unit = { - /* - * CoGroup with multiple key extractors - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets - .get5TupleDataSet(env) - val ds2 = CollectionDataSets.get3TupleDataSet(env) - val coGrouped = ds1.coGroup(ds2).where(t => (t._1, t._5)).equalTo(t => (t._1, t._2)) - .apply { - (first, second, out: Collector[(Int, Long, String)]) => - val strs = first map(_._4) - for (t <- second) { - for (s <- strs) { - out.collect((t._1, t._2, s)) - } - } - } - - coGrouped.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "1,1,Hallo\n" + "2,2,Hallo Welt\n" + "3,2,Hallo Welt wie gehts?\n" + "3,2," + - "ABC\n" + "5,3,HIJ\n" + "5,3,IJK\n" - - } - - @Test - def testCoGroupOnTwoCustomTypesUsingExpressionKeys(): Unit = { - /* - * CoGroup on two custom type inputs using expression keys - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val ds2 = CollectionDataSets.getCustomTypeDataSet(env) - val coGroupDs = ds.coGroup(ds2).where("myInt").equalTo("myInt").projecting { - (first, second) => - val o = new CustomType(0, 0, "test") - for (t <- first) { - o.myInt = t.myInt - o.myLong += t.myLong - } - for (t <- second) { - o.myInt = t.myInt - o.myLong += t.myLong - } - o - } - coGroupDs.writeAsText(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expectedResult = "1,0,test\n" + "2,6,test\n" + "3,24,test\n" + "4,60,test\n" + "5,120,test\n" + - "6," + "210,test\n" - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala deleted file mode 100644 index 3af53b14db2ec..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFCrossITCase.scala +++ /dev/null @@ -1,151 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.common.functions.RichCrossFunction -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.util.CollectionDataSets.CustomType -import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.{After, Before, Rule, Test} - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class AcceptPFCrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - - @Test - def testCorrectnessOfCrossOnTwoTupleInputs(): Unit = { - /* - * check correctness of cross on two tuple inputs - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) - val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds1.cross(ds2).projecting { - case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) - } - crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - - expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + - "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + - "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + - "Hallo Welt wieHallo Welt wie\n" - } - - @Test - def testCorrectnessOfCrossIfUDFReturnsLeftInput(): Unit = { - /* - * check correctness of cross if UDF returns left input object - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds1.cross(ds2).projecting { - case (l @ (_, _, _), _) => l - } - crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - - expected = "1,1,Hi\n" + "1,1,Hi\n" + "1,1,Hi\n" + "2,2,Hello\n" + "2,2,Hello\n" + "2,2," + - "Hello\n" + "3,2,Hello world\n" + "3,2,Hello world\n" + "3,2,Hello world\n" - } - - @Test - def testCorrectnessOfCrossIfUDFReturnsRightInput(): Unit = { - /* - * check correctness of cross if UDF returns right input object - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds1.cross(ds2).projecting { - case (_, r @ (_, _, _, _, _)) => r - } - crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - - expected = "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt," + - "2\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n" + "2,3,2,Hallo Welt wie," + - "1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,3,2,Hallo Welt wie,1\n" - } - - @Test - def testCorrectnessOfCrossWithHuge(): Unit = { - /* - * check correctness of crossWithHuge (only correctness of result -> should be the same - * as with normal cross) - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) - val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds1.crossWithHuge(ds2).projecting { - case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) - } - crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + - "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + - "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + - "Hallo Welt wieHallo Welt wie\n" - } - - @Test - def testCorrectnessOfCrossWithTiny(): Unit = { - /* - * check correctness of crossWithTiny (only correctness of result -> should be the same - * as with normal cross) - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall5TupleDataSet(env) - val ds2 = CollectionDataSets.getSmall5TupleDataSet(env) - val crossDs = ds1.crossWithTiny(ds2).projecting { - case ((_, _, l3, l4, _), (_, _, r3, r4, _)) => (l3 + r3, l4 + r4) - } - crossDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1," + - "Hallo WeltHallo\n" + "2,Hallo WeltHallo Welt\n" + "3,Hallo WeltHallo Welt wie\n" + "2," + - "Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4," + - "Hallo Welt wieHallo Welt wie\n" - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala deleted file mode 100644 index b1d8596ef4848..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFilterITCase.scala +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit._ -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -@RunWith(classOf[Parameterized]) -class AcceptPFFilterITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - - @Test - def testAllRejectingFilter(): Unit = { - /* - * Test all-rejecting filter. - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val filterDs = ds.filterWith( t => false ) - filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "\n" - } - - @Test - def testAllPassingFilter(): Unit = { - /* - * Test all-passing filter. - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val filterDs = ds.filterWith( t => true ) - filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + - "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + - "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + - "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + - "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + - "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - } - - @Test - def testFilterOnStringTupleField(): Unit = { - /* - * Test filter on String tuple field. - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val filterDs = ds.filterWith( _._3.contains("world") ) - filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" - } - - @Test - def testFilterOnIntegerTupleField(): Unit = { - /* - * Test filter on Integer tuple field. - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val filterDs = ds.filterWith( _._1 % 2 == 0 ) - filterDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," + - "Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," + - "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n" - } - - @Test - def testFilterBasicType(): Unit = { - /* - * Test filter on basic type - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getStringDataSet(env) - val filterDs = ds.filterWith( _.startsWith("H") ) - filterDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" - } - - @Test - def testFilterOnCustomType(): Unit = { - /* - * Test filter on custom type - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val filterDs = ds.filterWith( _.myString.contains("a") ) - filterDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala deleted file mode 100644 index 4385d2d88076b..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFFlatMapITCase.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit._ -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -@RunWith(classOf[Parameterized]) -class AcceptPFFlatMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - - @Test - def testNonPassingFlatMap(): Unit = { - /* - * Test non-passing flatmap - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getStringDataSet(env) - val nonPassingFlatMapDs = ds.flatMapWith( in => if (in.contains("banana")) Some(in) else None ) - nonPassingFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "\n" - } - - @Test - def testDataDuplicatingFlatMap(): Unit = { - /* - * Test data duplicating flatmap - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getStringDataSet(env) - val duplicatingFlatMapDs = ds.flatMapWith( in => Seq(in, in.toUpperCase) ) - duplicatingFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "Hi\n" + "HI\n" + "Hello\n" + "HELLO\n" + "Hello world\n" + "HELLO WORLD\n" + - "Hello world, how are you?\n" + "HELLO WORLD, HOW ARE YOU?\n" + "I am fine.\n" + "I AM " + - "FINE.\n" + "Luke Skywalker\n" + "LUKE SKYWALKER\n" + "Random comment\n" + "RANDOM " + - "COMMENT\n" + "LOL\n" + "LOL\n" - } - - @Test - def testFlatMapWithVaryingNumberOfEmittedTuples(): Unit = { - /* - * Test flatmap with varying number of emitted tuples - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val varyingTuplesMapDs = ds.flatMapWith { - case in @ (i, l, s) => - val numTuples = i % 3 - (0 until numTuples) map { i => in } - } - varyingTuplesMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "2,2,Hello\n" + "4,3,Hello world, " + - "how are you?\n" + "5,3,I am fine.\n" + "5,3,I am fine.\n" + "7,4,Comment#1\n" + "8,4," + - "Comment#2\n" + "8,4,Comment#2\n" + "10,4,Comment#4\n" + "11,5,Comment#5\n" + "11,5," + - "Comment#5\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "14,5,Comment#8\n" + "16,6," + - "Comment#10\n" + "17,6,Comment#11\n" + "17,6,Comment#11\n" + "19,6,Comment#13\n" + "20," + - "6,Comment#14\n" + "20,6,Comment#14\n" - } - - @Test - def testTypeConversionFlatMapperCustomToTuple(): Unit = { - /* - * Test type conversion flatmapper (Custom -> Tuple) - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val typeConversionFlatMapDs = ds.flatMapWith { in => Some((in.myInt, in.myLong, in.myString)) } - typeConversionFlatMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, " + - "how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + "4,6," + - "Comment#1\n" + "4,7,Comment#2\n" + "4,8,Comment#3\n" + "4,9,Comment#4\n" + "5,10," + - "Comment#5\n" + "5,11,Comment#6\n" + "5,12,Comment#7\n" + "5,13,Comment#8\n" + "5,14," + - "Comment#9\n" + "6,15,Comment#10\n" + "6,16,Comment#11\n" + "6,17,Comment#12\n" + "6," + - "18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n" - } - - @Test - def testTypeConversionFlatMapperTupleToBasic(): Unit = { - /* - * Test type conversion flatmapper (Tuple -> Basic) - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val typeConversionFlatMapDs = ds.flatMapWith { case (_, _, in) => Some(in) } - typeConversionFlatMapDs.writeAsText(resultPath, WriteMode.OVERWRITE) - env.execute() - expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine" + - ".\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + "Comment#3\n" + - "Comment#4\n" + "Comment#5\n" + "Comment#6\n" + "Comment#7\n" + "Comment#8\n" + - "Comment#9\n" + "Comment#10\n" + "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + - "Comment#14\n" + "Comment#15\n" - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala deleted file mode 100644 index bfdfe6332637b..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFMapITCase.scala +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.{ExecutionEnvironment, createTypeInformation} -import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.optimizer.Optimizer -import org.apache.flink.optimizer.dag.{MapNode, OptimizerNode} -import org.apache.flink.optimizer.plan.{PlanNode, SinkPlanNode, SourcePlanNode} -import org.apache.flink.util.TestLogger -import org.junit.Assert.assertTrue -import org.junit._ - -import scala.collection.JavaConversions._ - -class AcceptPFMapITCase extends TestLogger { - - private val optimizer = new Optimizer(new Configuration) - private val unusedResultPath = "UNUSED" - - def isTransformation(n: PlanNode): Boolean = - !n.isInstanceOf[SourcePlanNode] && !n.isInstanceOf[SinkPlanNode] - - private def getOptimizerTransformationNodes(env: ExecutionEnvironment): Iterable[OptimizerNode] = - for { - node <- optimizer.compile(env.createProgramPlan("UNUSED")).getAllNodes - transformation = node.getOptimizerNode if isTransformation(node) - } yield transformation - - @Test - def testIdentityMapperWithBasicType(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getStringDataSet(env) - val identityMapDs = ds.mapWith(identity) - identityMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - - @Test - def testIdentityMapperWithTuple(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val identityMapDs = ds.mapWith(identity) - identityMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - - @Test - def testTypeConversionMapperCustomToTuple(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val typeConversionMapDs = ds.mapWith( c => (c.myInt, c.myLong, c.myString) ) - typeConversionMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - - @Test - def testTypeConversionMapperTupleToBasic(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val typeConversionMapDs = ds.mapWith { case (_, _, a) => a } - typeConversionMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - - @Test - def testMapperOnTupleIncrementFieldReorderSecondAndThirdFields(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val tupleMapDs = ds.mapWith { case (a, b, c) => (a + 1, c, b) } - tupleMapDs.writeAsCsv(unusedResultPath, writeMode = WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - - @Test - def testMapperOnCustomLowercaseString(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val customMapDs = ds.mapWith { c => c.myString = c.myString.toLowerCase; c } - customMapDs.writeAsText(unusedResultPath, WriteMode.OVERWRITE) - val nodes = getOptimizerTransformationNodes(env) - assertTrue("The plan should contain 1 transformation", nodes.size == 1) - assertTrue("The transformation should be a map", nodes.forall(_.isInstanceOf[MapNode])) - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala deleted file mode 100644 index 69378255b977a..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AcceptPFReduceITCase.scala +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.common.functions.RichReduceFunction -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 -import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.{After, Before, Rule, Test} - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class AcceptPFReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - - @Test - def testReduceOnTuplesWithKeyFieldSelector(): Unit = { - /* - * Reduce on tuples with key field selector - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val reduceDs = ds.groupBy(1) - .reduceWith { case ((i1, l1, _), (i2, _, _)) => (i1 + i2, l1, "B-)") } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n" - } - - @Test - def testReduceOnTuplesWithMultipleKeyFieldSelectors(): Unit = { - /* - * Reduce on tuples with multiple key field selectors - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val reduceDs = ds.groupBy(4, 0) - .reduceWith { - case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => (in11, in12 + in22, 0, "P-)", in15) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + - "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + - "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" - } - - @Test - def testReduceOnTuplesWithKeyExtractor(): Unit = { - /* - * Reduce on tuples with key extractor - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val reduceDs = ds.groupBy(_._2) - .reduceWith { case ((i1, l1, _), (i2, _, _)) => (i1 + i2, l1, "B-)") } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n" - } - - @Test - def testAllReduceForTuple(): Unit = { - /* - * All-reduce for tuple - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get3TupleDataSet(env) - val reduceDs = - ds.reduceWith { case ((i1, l1, _), (i2, l2, _)) => (i1 + i2, l1 + l2, "Hello World") } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "231,91,Hello World\n" - } - - @Test - def testReduceWithATupleReturningKeySelector(): Unit = { - /* - * Reduce with a Tuple-returning KeySelector - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val reduceDs = ds.groupBy(t => (t._1, t._5)) - .reduceWith { case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => - (in11, in12 + in22, 0, "P-)", in15) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + - "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + - "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" - } - - @Test - def testReduceOnGroupedDSByExpressionKey(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.get5TupleDataSet(env) - val reduceDs = ds.groupBy("_5", "_1") - .reduceWith { case ((in11, in12, _, _, in15), (_, in22, _, _, _)) => - (in11, in12 + in22, 0, "P-)", in15) } - reduceDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0," + - "P-),2\n" + "3,6,5,BCD,3\n" + "4,17,0,P-),1\n" + "4,17,0,P-),2\n" + "5,11,10,GHI," + - "1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n" - } - -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala deleted file mode 100644 index 212e69db6d939..0000000000000 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/extensions/AccetPFJoinITCase.scala +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.api.scala.extensions - -import org.apache.flink.api.common.functions.RichJoinFunction -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit.rules.TemporaryFolder -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.{After, Before, Rule, Test} - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class AccetPFJoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - private var resultPath: String = null - private var expected: String = null - private val _tempFolder = new TemporaryFolder() - - @Rule - def tempFolder = _tempFolder - - @Before - def before(): Unit = { - resultPath = tempFolder.newFile().toURI.toString - } - - @After - def after(): Unit = { - TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath) - } - - @Test - def testUDFJoinOnTuplesWithKeyFieldPositions(): Unit = { - /* - * UDF Join on tuples with key field positions - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { - case ((_, _, l), (_, _, _, r, _)) => l -> r - } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" - } - - @Test - def testUDFJoinOnTuplesWithMultipleKeyFieldPositions(): Unit = { - /* - * UDF Join on tuples with multiple key field positions - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.get3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where(0, 1).equalTo(0, 4).projecting { - case ((_, _, l), (_, _, _, r, _)) => l -> r - } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + - "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n" - } - - @Test - def testJoinWithHuge(): Unit = { - /* - * Join with Huge - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.joinWithHuge(ds2).where(1).equalTo(1).projecting { - case ((_, _, l), (_, _, _, r, _)) => l -> r - } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" - } - - @Test - def testJoinWithTiny(): Unit = { - /* - * Join with Tiny - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.joinWithTiny(ds2).where(1).equalTo(1).projecting { - case ((_, _, l), (_, _, _, r, _)) => l -> r - } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" - } - - @Test - def testJoinThatReturnsTheLeftInputObject(): Unit = { - /* - * Join that returns the left input object - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (l @ (_, _, _), _) => l } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" - } - - @Test - def testJoinThatReturnsTheRightInputObject(): Unit = { - /* - * Join that returns the right input object - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where(1).equalTo(1).projecting { case (_, r @ (_, _, _, _, _)) => r } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n" - } - - @Test - def testUDFJoinOnTuplesWithTupleReturningKeySelectors(): Unit = { - /* - * UDF Join on tuples with tuple-returning key selectors - */ - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val ds1 = CollectionDataSets.get3TupleDataSet(env) - val ds2 = CollectionDataSets.get5TupleDataSet(env) - val joinDs = ds1.join(ds2).where( t => (t._1, t._2)).equalTo( t => (t._1, t._5)).projecting { - case ((_, _, l), (_, _, _, r, _)) => l -> r - } - joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + - "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n" - } - -} From ab66a3c70a8dc1e31afaf7cc4b3b4955ecb8eea5 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 17:13:52 +0100 Subject: [PATCH 13/39] [FLINK-1159] Shared enviroment and fixture in base class --- .../AcceptPartialFunctionsTestBase.scala | 14 ++++++++++++++ .../acceptPartialFunctions/OnDataSetTest.scala | 8 +------- 2 files changed, 15 insertions(+), 7 deletions(-) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala new file mode 100644 index 0000000000000..f083167e628f6 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala @@ -0,0 +1,14 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.util.TestLogger +import org.scalatest.junit.JUnitSuiteLike + +abstract class AcceptPartialFunctionsTestBase extends TestLogger with JUnitSuiteLike { + + private val env = ExecutionEnvironment.getExecutionEnvironment + + protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") + protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala index e1fb4f6a4d2af..6ff0f5cf44cea 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala @@ -21,15 +21,9 @@ package org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnDataSet -import org.apache.flink.util.TestLogger import org.junit.Test -import org.scalatest.junit.JUnitSuiteLike -class OnDataSetTest extends TestLogger with JUnitSuiteLike { - - private val env = ExecutionEnvironment.getExecutionEnvironment - private val tuples = env.fromElements(1 -> "hello", 2 -> "world") - private val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) +class OnDataSetTest extends AcceptPartialFunctionsTestBase { @Test def testMapWithOnTuple(): Unit = { From 401058e9b26379f1e7e08011752835d147fde2a6 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 17:51:05 +0100 Subject: [PATCH 14/39] [FLINK-1159] Add grouped fixtures --- .../AcceptPartialFunctionsTestBase.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala index f083167e628f6..5a1b722a967e6 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala @@ -11,4 +11,7 @@ abstract class AcceptPartialFunctionsTestBase extends TestLogger with JUnitSuite protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + protected val groupedTuples = tuples.groupBy(_._1) + protected val groupedCaseObjects = caseObjects.groupBy(_.id) + } From 3119086f6136c0c04b2e59a466d027ba879618b7 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 24 Mar 2016 17:51:23 +0100 Subject: [PATCH 15/39] [FLINK-1159] Add OnGroupedDataSet tests --- .../OnGroupedDataSetTest.scala | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala new file mode 100644 index 0000000000000..6d33faf19a048 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -0,0 +1,101 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.operators.Order +import org.apache.flink.api.java.operators.{GroupCombineOperator, GroupReduceOperator, ReduceOperator} +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnGroupedDataSet +import org.junit.{Ignore, Test} + +class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { + + @Test + def testSortGroupWithOnTuple(): Unit = { + val test = + groupedTuples.sortGroupWith(Order.ASCENDING) { + case (id, _) => id + } + assert(test.isInstanceOf[GroupedDataSet[_]], + "sortGroupWith should produce a GroupedDataSet") + } + + @Test + def testSortGroupWithOnCaseClass(): Unit = { + val test = + groupedCaseObjects.sortGroupWith(Order.ASCENDING) { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[GroupedDataSet[_]], + "sortGroupWith should produce a GroupedDataSet") + } + + @Test + def testReduceWithOnTuple(): Unit = { + val test = + groupedTuples.reduceWith { + case ((_, v1), (_, v2)) => (0, s"$v1 $v2") + } + + assert(test.javaSet.isInstanceOf[ReduceOperator[_]], + "reduceWith should produce a ReduceOperator") + } + + @Test + def testReduceWithOnCaseClass(): Unit = { + val test = + groupedCaseObjects.reduceWith { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => KeyValuePair(0, s"$v1 $v2") + } + + assert(test.javaSet.isInstanceOf[ReduceOperator[_]], + "reduceWith should produce a ReduceOperator") + } + + @Test + def testReduceGroupWithOnTuple(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + groupedTuples.reduceGroupWith { + case (_, value) #:: _ => accumulator.append(value).append('\n') + } + + assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], + "reduceGroupWith should produce a GroupReduceOperator") + } + + @Test + def testReduceGroupWithOnCaseClass(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + groupedCaseObjects.reduceGroupWith { + case KeyValuePair(_, value) #:: _ => accumulator.append(value).append('\n') + } + + assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], + "reduceGroupWith should produce a GroupReduceOperator") + } + + @Test + def testCombineGroupWithOnTuple(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + groupedTuples.combineGroupWith { + case (_, value) #:: _ => accumulator.append(value).append('\n') + } + + assert(test.javaSet.isInstanceOf[GroupCombineOperator[_, _]], + "combineGroupWith should produce a GroupCombineOperator") + } + + @Test + def testCombineGroupWithOnCaseClass(): Unit = { + val accumulator: StringBuffer = new StringBuffer() + val test = + groupedCaseObjects.combineGroupWith { + case KeyValuePair(_, value) #:: _ => accumulator.append(value).append('\n') + } + + assert(test.javaSet.isInstanceOf[GroupCombineOperator[_, _]], + "combineGroupWith should produce a GroupCombineOperator") + } + +} From e199b63e3e9cc681e3edc301cc0ae83f7418ac31 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 12:56:21 +0100 Subject: [PATCH 16/39] [FLINK-1159] Moved helpers to different packages --- .../extensions/acceptPartialFunctions/OnDataSetTest.scala | 2 ++ .../acceptPartialFunctions/OnGroupedDataSetTest.scala | 4 +++- .../{ => base}/AcceptPartialFunctionsTestBase.scala | 3 ++- .../acceptPartialFunctions/{ => data}/KeyValuePair.scala | 2 +- 4 files changed, 8 insertions(+), 3 deletions(-) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/{ => base}/AcceptPartialFunctionsTestBase.scala (86%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/{ => data}/KeyValuePair.scala (96%) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala index 6ff0f5cf44cea..c9f4938e2fb18 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala @@ -20,6 +20,8 @@ package org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnDataSet import org.junit.Test diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala index 6d33faf19a048..2e357f50a0ab1 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -3,8 +3,10 @@ package org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.operators.{GroupCombineOperator, GroupReduceOperator, ReduceOperator} import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnGroupedDataSet -import org.junit.{Ignore, Test} +import org.junit.Test class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala similarity index 86% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala index 5a1b722a967e6..9d93dd537437e 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/AcceptPartialFunctionsTestBase.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala @@ -1,6 +1,7 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.acceptPartialFunctions.base import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair import org.apache.flink.util.TestLogger import org.scalatest.junit.JUnitSuiteLike diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala similarity index 96% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala index b20b0d2df6efa..fe70eb788084e 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/KeyValuePair.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala @@ -1,4 +1,4 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.acceptPartialFunctions.data /** * Simple case class to test the `acceptPartialFunctions` extension From 6becaf793e0203d8c2b4165401c50b68ae27ffe0 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 14:11:35 +0100 Subject: [PATCH 17/39] [FLINK-1159] Slight test hints renaming + added extensions for key-pair ops --- .../OnHalfUnfinishedKeyPairOperation.scala | 11 ++ .../OnUnfinishedKeyPairOperation.scala | 11 ++ .../flink/api/scala/extensions/package.scala | 164 +----------------- .../OnGroupedDataSetTest.scala | 16 +- 4 files changed, 39 insertions(+), 163 deletions(-) create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala new file mode 100644 index 0000000000000..5288eea5025db --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala @@ -0,0 +1,11 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation + +class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation](ds: HalfUnfinishedKeyPairOperation[L, R, O]) { + + def isEqualTo[K: TypeInformation](fun: R => K): O = + ds.equalTo(fun) + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala new file mode 100644 index 0000000000000..fe8b485f44f5d --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala @@ -0,0 +1,11 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.{HalfUnfinishedKeyPairOperation, UnfinishedKeyPairOperation} + +class OnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation](ds: UnfinishedKeyPairOperation[L, R, O]) { + + def whereClause[K: TypeInformation](fun: (L) => K): HalfUnfinishedKeyPairOperation[L, R, O] = + ds.where(fun) + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index 81f3743fe0f80..1a8fbae00ae1a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -24,178 +24,32 @@ import scala.reflect.ClassTag package object extensions { - /** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnDataSet[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = new OnDataSet[T](ds) - /** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnJoinDataSet[L: TypeInformation, R: TypeInformation]( ds: JoinDataSet[L, R]): OnJoinDataSet[L, R] = new OnJoinDataSet[L, R](ds) - /** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnCrossDataSet[L: TypeInformation, R: TypeInformation]( ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = new OnCrossDataSet[L, R](ds) - /** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnGroupedDataSet[T: TypeInformation: ClassTag]( ds: GroupedDataSet[T]): OnGroupedDataSet[T] = new OnGroupedDataSet[T](ds) - /** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ + implicit def acceptPartialFunctionsOnCoGroupDataSet[L: TypeInformation, R: TypeInformation]( ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = new OnCoGroupDataSet[L, R](ds) + implicit def acceptPartialFunctionsOnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + ds: HalfUnfinishedKeyPairOperation[L, R, O]): OnHalfUnfinishedKeyPairOperation[L, R, O] = + new OnHalfUnfinishedKeyPairOperation[L, R, O](ds) + + implicit def acceptPartialFunctionsOnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + ds: UnfinishedKeyPairOperation[L, R, O]): OnUnfinishedKeyPairOperation[L, R, O] = + new OnUnfinishedKeyPairOperation[L, R, O](ds) + } diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala index 2e357f50a0ab1..af046d634ec7b 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -17,7 +17,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { case (id, _) => id } assert(test.isInstanceOf[GroupedDataSet[_]], - "sortGroupWith should produce a GroupedDataSet") + "sortGroupWith on tuples should produce a GroupedDataSet") } @Test @@ -27,7 +27,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[GroupedDataSet[_]], - "sortGroupWith should produce a GroupedDataSet") + "sortGroupWith on case objects should produce a GroupedDataSet") } @Test @@ -38,7 +38,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[ReduceOperator[_]], - "reduceWith should produce a ReduceOperator") + "reduceWith on tuples should produce a ReduceOperator") } @Test @@ -49,7 +49,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[ReduceOperator[_]], - "reduceWith should produce a ReduceOperator") + "reduceWith on case objects should produce a ReduceOperator") } @Test @@ -61,7 +61,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], - "reduceGroupWith should produce a GroupReduceOperator") + "reduceGroupWith on tuples should produce a GroupReduceOperator") } @Test @@ -73,7 +73,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[GroupReduceOperator[_, _]], - "reduceGroupWith should produce a GroupReduceOperator") + "reduceGroupWith on case objects should produce a GroupReduceOperator") } @Test @@ -85,7 +85,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[GroupCombineOperator[_, _]], - "combineGroupWith should produce a GroupCombineOperator") + "combineGroupWith on tuples should produce a GroupCombineOperator") } @Test @@ -97,7 +97,7 @@ class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { } assert(test.javaSet.isInstanceOf[GroupCombineOperator[_, _]], - "combineGroupWith should produce a GroupCombineOperator") + "combineGroupWith on case objects should produce a GroupCombineOperator") } } From 4f8cc2bc406761277fdd5a1e7ec1bd46d5b1c9c1 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 14:12:07 +0100 Subject: [PATCH 18/39] [FLINK-1159] Add tests for OnCoGroupDataSet and key-pair ops --- .../OnCoGroupDataSetTest.scala | 40 ++++++++++++ ...OnHalfUnfinishedKeyPairOperationTest.scala | 61 +++++++++++++++++++ .../OnUnfinishedKeyPairOperationTest.scala | 51 ++++++++++++++++ 3 files changed, 152 insertions(+) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala new file mode 100644 index 0000000000000..521bba0d42504 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -0,0 +1,40 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.java.operators.CoGroupOperator +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions._ +import org.junit.Test + +class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { + + @Test + def testProjectingOnTuple(): Unit = { + val test = + tuples.coGroup(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + }.projecting { + case ((id1, val1) #:: _, (id2, val2) #:: _) => s"$val1 $val2" + } + assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], + "projecting on tuples should produce a CoGroupOperator") + } + + @Test + def testProjectingOnCaseClass(): Unit = { + val test = + caseObjects.coGroup(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + }.projecting { + case (KeyValuePair(id1, val1) #:: _, KeyValuePair(id2, val2) #:: _) => s"$val1 $val2" + } + assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], + "projecting on case objects should produce a CoGroupOperator") + } + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala new file mode 100644 index 0000000000000..15e5fc57f502a --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala @@ -0,0 +1,61 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.java.operators.CoGroupOperator +import org.apache.flink.api.java.operators.JoinOperator.EquiJoin +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.junit.Test + +class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { + + @Test + def testJoinIsEqualToOnTuple(): Unit = { + val test = + tuples.join(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "isEqualTo for join on tuples should produce a EquiJoin") + } + + @Test + def testJoinIsEqualToOnCaseClass(): Unit = { + val test = + caseObjects.join(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "isEqualTo for join on case objects should produce a EquiJoin") + } + + @Test + def testCoGroupIsEqualToOnTuple(): Unit = { + val test = + tuples.coGroup(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + } + assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], + "isEqualTo for coGroup on tuples should produce a CoGroupOperator") + } + + @Test + def testCoGroupIsEqualToOnCaseClass(): Unit = { + val test = + caseObjects.coGroup(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + } + assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], + "isEqualTo for coGroup on case objects should produce a CoGroupOperator") + } + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala new file mode 100644 index 0000000000000..867f6f9dc6728 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala @@ -0,0 +1,51 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.junit.Test + +class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { + + @Test + def testJoinWhereClauseOnTuple(): Unit = { + val test = + tuples.join(tuples).whereClause { + case (id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for join on tuples should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testJoinWhereClauseOnCaseClass(): Unit = { + val test = + caseObjects.join(caseObjects).whereClause { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for join on case objects should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testCoGroupWhereClauseOnTuple(): Unit = { + val test = + tuples.coGroup(tuples).whereClause { + case (id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for coGroup on tuples should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testCoGroupWhereClauseOnCaseClass(): Unit = { + val test = + caseObjects.coGroup(caseObjects).whereClause { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for coGroup on case objects should produce a HalfUnfinishedKeyPairOperation") + } + +} From 4515bd11a609874fabf44d3c1418b07e0a9d0874 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 14:17:13 +0100 Subject: [PATCH 19/39] [FLINK-1159] Ignore unused data --- .../acceptPartialFunctions/OnCoGroupDataSetTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala index 521bba0d42504..d05edd6d83a85 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -17,7 +17,7 @@ class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { }.isEqualTo { case (id, _) => id }.projecting { - case ((id1, val1) #:: _, (id2, val2) #:: _) => s"$val1 $val2" + case ((_, v1) #:: _, (_, v2) #:: _) => s"$v1 $v2" } assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], "projecting on tuples should produce a CoGroupOperator") @@ -31,7 +31,7 @@ class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { }.isEqualTo { case KeyValuePair(id, _) => id }.projecting { - case (KeyValuePair(id1, val1) #:: _, KeyValuePair(id2, val2) #:: _) => s"$val1 $val2" + case (KeyValuePair(_, v1) #:: _, KeyValuePair(_, v2) #:: _) => s"$v1 $v2" } assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], "projecting on case objects should produce a CoGroupOperator") From 3089969f2d20d99d3e9630a0492423e7c2995ee3 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 14:43:29 +0100 Subject: [PATCH 20/39] [FLINK-1159] Add support for outer joins (with tests) --- ...Set.scala => OnJoinFunctionAssigner.scala} | 4 +- .../flink/api/scala/extensions/package.scala | 6 +- .../OnCoGroupDataSetTest.scala | 4 +- ...OnHalfUnfinishedKeyPairOperationTest.scala | 84 +++++++++++- .../OnJoinFunctionAssignerTest.scala | 124 ++++++++++++++++++ .../OnUnfinishedKeyPairOperationTest.scala | 72 +++++++++- 6 files changed, 275 insertions(+), 19 deletions(-) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/{OnJoinDataSet.scala => OnJoinFunctionAssigner.scala} (88%) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala similarity index 88% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala index a2ae49c959a93..15a3f51d9cc49 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala @@ -18,11 +18,11 @@ package org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.{DataSet, JoinDataSet} +import org.apache.flink.api.scala.{DataSet, JoinFunctionAssigner} import scala.reflect.ClassTag -class OnJoinDataSet[L: TypeInformation, R: TypeInformation](dataset: JoinDataSet[L, R]) { +class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation](dataset: JoinFunctionAssigner[L, R]) { /** * Joins the data sets using the function `fun` to project elements from both in the diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index 1a8fbae00ae1a..83011817d50e1 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -27,9 +27,9 @@ package object extensions { implicit def acceptPartialFunctionsOnDataSet[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = new OnDataSet[T](ds) - implicit def acceptPartialFunctionsOnJoinDataSet[L: TypeInformation, R: TypeInformation]( - ds: JoinDataSet[L, R]): OnJoinDataSet[L, R] = - new OnJoinDataSet[L, R](ds) + implicit def acceptPartialFunctionsOnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation]( + ds: JoinFunctionAssigner[L, R]): OnJoinFunctionAssigner[L, R] = + new OnJoinFunctionAssigner[L, R](ds) implicit def acceptPartialFunctionsOnCrossDataSet[L: TypeInformation, R: TypeInformation]( ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala index d05edd6d83a85..52599507c9200 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -10,7 +10,7 @@ import org.junit.Test class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { @Test - def testProjectingOnTuple(): Unit = { + def testCoGroupProjectingOnTuple(): Unit = { val test = tuples.coGroup(tuples).whereClause { case (id, _) => id @@ -24,7 +24,7 @@ class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { } @Test - def testProjectingOnCaseClass(): Unit = { + def testCoGroupProjectingOnCaseClass(): Unit = { val test = caseObjects.coGroup(caseObjects).whereClause { case KeyValuePair(id, _) => id diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala index 15e5fc57f502a..ef6d6a1fe914f 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala @@ -11,7 +11,7 @@ import org.junit.Test class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { @Test - def testJoinIsEqualToOnTuple(): Unit = { + def testInnerJoinIsEqualToOnTuple(): Unit = { val test = tuples.join(tuples).whereClause { case (id, _) => id @@ -19,11 +19,11 @@ class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBas case (id, _) => id } assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], - "isEqualTo for join on tuples should produce a EquiJoin") + "isEqualTo for inner join on tuples should produce a EquiJoin") } @Test - def testJoinIsEqualToOnCaseClass(): Unit = { + def testInnerJoinIsEqualToOnCaseClass(): Unit = { val test = caseObjects.join(caseObjects).whereClause { case KeyValuePair(id, _) => id @@ -31,7 +31,79 @@ class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBas case KeyValuePair(id, _) => id } assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], - "isEqualTo for join on case objects should produce a EquiJoin") + "isEqualTo for inner join on case objects should produce a EquiJoin") + } + + @Test + def testRightOuterJoinIsEqualToOnTuple(): Unit = { + val test = + tuples.rightOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for right outer join on tuples should produce a JoinFunctionAssigner") + } + + @Test + def testRightOuterJoinIsEqualToOnCaseClass(): Unit = { + val test = + caseObjects.rightOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for right outer join on case objects should produce a JoinFunctionAssigner") + } + + @Test + def testLeftOuterJoinIsEqualToOnTuple(): Unit = { + val test = + tuples.leftOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for left outer join on tuples should produce a JoinFunctionAssigner") + } + + @Test + def testLeftOuterJoinIsEqualToOnCaseClass(): Unit = { + val test = + caseObjects.leftOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for left outer join on case objects should produce a JoinFunctionAssigner") + } + + @Test + def testFullOuterJoinIsEqualToOnTuple(): Unit = { + val test = + tuples.fullOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for full outer join on tuples should produce a JoinFunctionAssigner") + } + + @Test + def testFullOuterJoinIsEqualToOnCaseClass(): Unit = { + val test = + caseObjects.fullOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[JoinFunctionAssigner[_, _]], + "isEqualTo for full outer join on case objects should produce a JoinFunctionAssigner") } @Test @@ -43,7 +115,7 @@ class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBas case (id, _) => id } assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], - "isEqualTo for coGroup on tuples should produce a CoGroupOperator") + "isEqualTo for co-group on tuples should produce a CoGroupOperator") } @Test @@ -55,7 +127,7 @@ class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBas case KeyValuePair(id, _) => id } assert(test.javaSet.isInstanceOf[CoGroupOperator[_, _, _]], - "isEqualTo for coGroup on case objects should produce a CoGroupOperator") + "isEqualTo for co-group on case objects should produce a CoGroupOperator") } } diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala new file mode 100644 index 0000000000000..512ef9894d392 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala @@ -0,0 +1,124 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.java.operators.JoinOperator.EquiJoin +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.junit.Test + +class OnJoinFunctionAssignerTest extends AcceptPartialFunctionsTestBase { + + @Test + def testInnerJoinProjectingOnTuple(): Unit = { + val test = + tuples.join(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + }.projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting inner join on tuples should produce a EquiJoin") + } + + @Test + def testInnerJoinProjectingOnCaseClass(): Unit = { + val test = + caseObjects.join(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + }.projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting inner join on case objects should produce a EquiJoin") + } + + @Test + def testRightOuterJoinProjectingOnTuple(): Unit = { + val test = + tuples.rightOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + }.projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting right outer join on tuples should produce a EquiJoin") + } + + @Test + def testRightOuterJoinProjectingOnCaseClass(): Unit = { + val test = + caseObjects.rightOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + }.projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting right outer join on case objects should produce a EquiJoin") + } + + @Test + def testLeftOuterJoinProjectingOnTuple(): Unit = { + val test = + tuples.leftOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + }.projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting left outer join on tuples should produce a EquiJoin") + } + + @Test + def testLeftOuterJoinProjectingOnCaseClass(): Unit = { + val test = + caseObjects.leftOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + }.projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting left outer join on case objects should produce a EquiJoin") + } + + @Test + def testFullOuterJoinProjectingOnTuple(): Unit = { + val test = + tuples.fullOuterJoin(tuples).whereClause { + case (id, _) => id + }.isEqualTo { + case (id, _) => id + }.projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting full outer join on tuples should produce a EquiJoin") + } + + @Test + def testFullOuterJoinProjectingOnCaseClass(): Unit = { + val test = + caseObjects.fullOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + }.isEqualTo { + case KeyValuePair(id, _) => id + }.projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[EquiJoin[_, _, _]], + "projecting full outer join on case objects should produce a EquiJoin") + } + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala index 867f6f9dc6728..a845b65c6b2de 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala @@ -9,23 +9,83 @@ import org.junit.Test class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { @Test - def testJoinWhereClauseOnTuple(): Unit = { + def testInnerJoinWhereClauseOnTuple(): Unit = { val test = tuples.join(tuples).whereClause { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for join on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for inner join on tuples should produce a HalfUnfinishedKeyPairOperation") } @Test - def testJoinWhereClauseOnCaseClass(): Unit = { + def testInnerJoinWhereClauseOnCaseClass(): Unit = { val test = caseObjects.join(caseObjects).whereClause { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for join on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for inner join on case objects should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testRightOuterJoinWhereClauseOnTuple(): Unit = { + val test = + tuples.join(tuples).whereClause { + case (id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for right outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testRightOuterJoinWhereClauseOnCaseClass(): Unit = { + val test = + caseObjects.join(caseObjects).whereClause { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for right outer join on case objects should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testLeftOuterJoinWhereClauseOnTuple(): Unit = { + val test = + tuples.join(tuples).whereClause { + case (id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for left outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testLeftOuterJoinWhereClauseOnCaseClass(): Unit = { + val test = + caseObjects.join(caseObjects).whereClause { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for left outer join on case objects should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testFullOuterJoinWhereClauseOnTuple(): Unit = { + val test = + tuples.fullOuterJoin(tuples).whereClause { + case (id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for full outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + } + + @Test + def testFullOuterJoinWhereClauseOnCaseClass(): Unit = { + val test = + caseObjects.fullOuterJoin(caseObjects).whereClause { + case KeyValuePair(id, _) => id + } + assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], + "whereClause for full outer join on case objects should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -35,7 +95,7 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for coGroup on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for co-group on tuples should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -45,7 +105,7 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for coGroup on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for co-group on case objects should produce a HalfUnfinishedKeyPairOperation") } } From 135013a02c060cd4beb5e5057a48058059462b39 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 15:21:57 +0100 Subject: [PATCH 21/39] [FLINK-1159] Add tests for OnCrossDataSet --- .../OnCrossDataSetTest.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala new file mode 100644 index 0000000000000..61a5c477e2a29 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala @@ -0,0 +1,32 @@ +package org.apache.flink.api.scala.extensions.acceptPartialFunctions + +import org.apache.flink.api.java.operators.CrossOperator +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.junit.Test + +class OnCrossDataSetTest extends AcceptPartialFunctionsTestBase { + + @Test + def testCrossProjectingOnTuple(): Unit = { + val test = + tuples.cross(tuples).projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[CrossOperator[_, _, _]], + "projecting for cross on tuples should produce a CrossOperator") + } + + @Test + def testCrossProjectingOnCaseClass(): Unit = { + val test = + caseObjects.cross(caseObjects).projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaSet.isInstanceOf[CrossOperator[_, _, _]], + "projecting for cross on case objects should produce a CrossOperator") + } + +} From de2ae5a97c323db491cdb2f9da58358f6d7bdb23 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 15:22:25 +0100 Subject: [PATCH 22/39] [FLINK-1159] Add package Scaladoc --- .../acceptPartialFunctions/package.scala | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala new file mode 100644 index 0000000000000..9b942b2d90eb4 --- /dev/null +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala @@ -0,0 +1,38 @@ +package org.apache.flink.api.scala.extensions + +/** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ +package object acceptPartialFunctions { + + // Empty: this package object purpose is just to document the package (see Scaladoc above) + +} From 3a520d59029e5a024890653ab1665e02ca1b5ade Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 16:40:54 +0100 Subject: [PATCH 23/39] [FLINK-1159] Refactoring to include all acceptPartialFunctions implementation with a single import --- .../acceptPartialFunctions/package.scala | 38 --------------- .../OnCoGroupDataSet.scala | 4 +- .../OnCrossDataSet.scala | 4 +- .../acceptPartialFunctions/OnDataSet.scala | 4 +- .../OnGroupedDataSet.scala | 2 +- .../OnHalfUnfinishedKeyPairOperation.scala | 2 +- .../OnJoinFunctionAssigner.scala | 2 +- .../OnUnfinishedKeyPairOperation.scala | 2 +- .../flink/api/scala/extensions/package.scala | 47 +++++++++++++++---- .../OnCoGroupDataSetTest.scala | 8 ++-- .../OnCrossDataSetTest.scala | 8 ++-- .../OnDataSetTest.scala | 8 ++-- .../OnGroupedDataSetTest.scala | 8 ++-- ...OnHalfUnfinishedKeyPairOperationTest.scala | 8 ++-- .../OnJoinFunctionAssignerTest.scala | 8 ++-- .../OnUnfinishedKeyPairOperationTest.scala | 8 ++-- .../base/AcceptPartialFunctionsTestBase.scala | 4 +- .../data/KeyValuePair.scala | 3 +- 18 files changed, 81 insertions(+), 87 deletions(-) delete mode 100644 flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnCoGroupDataSet.scala (91%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnCrossDataSet.scala (91%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnDataSet.scala (96%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnGroupedDataSet.scala (97%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala (82%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnJoinFunctionAssigner.scala (95%) rename flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala (84%) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala deleted file mode 100644 index 9b942b2d90eb4..0000000000000 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/package.scala +++ /dev/null @@ -1,38 +0,0 @@ -package org.apache.flink.api.scala.extensions - -/** - * acceptPartialFunctions extends the original DataSet with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = ExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.groupingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ -package object acceptPartialFunctions { - - // Empty: this package object purpose is just to document the package (see Scaladoc above) - -} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala similarity index 91% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala index d0ffef8696214..40fb729927c99 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.{DataSet, CoGroupDataSet} +import org.apache.flink.api.scala.{CoGroupDataSet, DataSet} import scala.reflect.ClassTag diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala similarity index 91% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala index 4f931f02de7b1..b0094f7c5e3d0 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.{DataSet, CrossDataSet} +import org.apache.flink.api.scala.{CrossDataSet, DataSet} import scala.reflect.ClassTag diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala similarity index 96% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala index c9b73117141a3..eb3c6187a3917 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.{GroupedDataSet, DataSet} +import org.apache.flink.api.scala.{DataSet, GroupedDataSet} import scala.reflect.ClassTag diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala similarity index 97% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala index 4bbe192823920..6252305a477a0 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.operators.Order import org.apache.flink.api.common.typeinfo.TypeInformation diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala similarity index 82% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala index 5288eea5025db..97bb4c7e99268 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala @@ -1,4 +1,4 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala similarity index 95% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala index 15a3f51d9cc49..53f28f15984de 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssigner.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{DataSet, JoinFunctionAssigner} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala similarity index 84% rename from flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala rename to flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala index fe8b485f44f5d..d64a0754a18b3 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala @@ -1,4 +1,4 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{HalfUnfinishedKeyPairOperation, UnfinishedKeyPairOperation} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index 83011817d50e1..ae9f24daa6c9d 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -18,37 +18,68 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.extensions.acceptPartialFunctions._ +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions._ import scala.reflect.ClassTag +/** + * acceptPartialFunctions extends the original DataSet with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * e.g. + * {{{ + * object Main { + * import org.apache.flink.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = ExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.groupingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + */ package object extensions { - implicit def acceptPartialFunctionsOnDataSet[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = + implicit def acceptPartialFunctions[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = new OnDataSet[T](ds) - implicit def acceptPartialFunctionsOnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: JoinFunctionAssigner[L, R]): OnJoinFunctionAssigner[L, R] = new OnJoinFunctionAssigner[L, R](ds) - implicit def acceptPartialFunctionsOnCrossDataSet[L: TypeInformation, R: TypeInformation]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = new OnCrossDataSet[L, R](ds) - implicit def acceptPartialFunctionsOnGroupedDataSet[T: TypeInformation: ClassTag]( + implicit def acceptPartialFunctions[T: TypeInformation: ClassTag]( ds: GroupedDataSet[T]): OnGroupedDataSet[T] = new OnGroupedDataSet[T](ds) - implicit def acceptPartialFunctionsOnCoGroupDataSet[L: TypeInformation, R: TypeInformation]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = new OnCoGroupDataSet[L, R](ds) - implicit def acceptPartialFunctionsOnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( ds: HalfUnfinishedKeyPairOperation[L, R, O]): OnHalfUnfinishedKeyPairOperation[L, R, O] = new OnHalfUnfinishedKeyPairOperation[L, R, O](ds) - implicit def acceptPartialFunctionsOnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( ds: UnfinishedKeyPairOperation[L, R, O]): OnUnfinishedKeyPairOperation[L, R, O] = new OnUnfinishedKeyPairOperation[L, R, O](ds) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala index 52599507c9200..8f9e3686442e0 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -1,10 +1,10 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CoGroupOperator import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair -import org.apache.flink.api.scala.extensions._ +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala index 61a5c477e2a29..1ed14fb43585e 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala @@ -1,10 +1,10 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CrossOperator import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.junit.Test class OnCrossDataSetTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala index c9f4938e2fb18..bcf63ca9a29dc 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair -import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnDataSet +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.junit.Test class OnDataSetTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala index af046d634ec7b..0e8aaa1c063a3 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -1,11 +1,11 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.operators.{GroupCombineOperator, GroupReduceOperator, ReduceOperator} import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair -import org.apache.flink.api.scala.extensions.acceptPartialFunctionsOnGroupedDataSet +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala index ef6d6a1fe914f..4ededf7a2bf9e 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala @@ -1,11 +1,11 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CoGroupOperator import org.apache.flink.api.java.operators.JoinOperator.EquiJoin import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.junit.Test class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala index 512ef9894d392..d8c84190dbbbf 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala @@ -1,10 +1,10 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.JoinOperator.EquiJoin import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.junit.Test class OnJoinFunctionAssignerTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala index a845b65c6b2de..24aa144f1e558 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala @@ -1,9 +1,9 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.junit.Test class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala index 9d93dd537437e..776d82004348d 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala @@ -1,7 +1,7 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions.base +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair import org.apache.flink.util.TestLogger import org.scalatest.junit.JUnitSuiteLike diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala index fe70eb788084e..982ce3a8da19e 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala @@ -1,7 +1,8 @@ -package org.apache.flink.api.scala.extensions.acceptPartialFunctions.data +package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data /** * Simple case class to test the `acceptPartialFunctions` extension + * * @param id A numerical identifier * @param value A textual value */ From aea171f05983bc3214b18b525697d3b8a555aed8 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:22:39 +0100 Subject: [PATCH 24/39] [FLINK-1159] Refactor packages layout --- .../base/AcceptPartialFunctionsTestBase.scala | 18 ---------- .../data/KeyValuePair.scala | 9 ----- .../extensions/base/AcceptPFTestBase.scala | 35 +++++++++++++++++++ .../scala/extensions/data/KeyValuePair.scala | 26 ++++++++++++++ 4 files changed, 61 insertions(+), 27 deletions(-) delete mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala delete mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala deleted file mode 100644 index 776d82004348d..0000000000000 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/base/AcceptPartialFunctionsTestBase.scala +++ /dev/null @@ -1,18 +0,0 @@ -package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair -import org.apache.flink.util.TestLogger -import org.scalatest.junit.JUnitSuiteLike - -abstract class AcceptPartialFunctionsTestBase extends TestLogger with JUnitSuiteLike { - - private val env = ExecutionEnvironment.getExecutionEnvironment - - protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") - protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) - - protected val groupedTuples = tuples.groupBy(_._1) - protected val groupedCaseObjects = caseObjects.groupBy(_.id) - -} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala deleted file mode 100644 index 982ce3a8da19e..0000000000000 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/data/KeyValuePair.scala +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data - -/** - * Simple case class to test the `acceptPartialFunctions` extension - * - * @param id A numerical identifier - * @param value A textual value - */ -private[acceptPartialFunctions] case class KeyValuePair(id: Int, value: String) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala new file mode 100644 index 0000000000000..ed5e2b8ea5514 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala @@ -0,0 +1,35 @@ +/* + * 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.api.scala.extensions.test.acceptPartialFunctions.base + +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.data.KeyValuePair +import org.apache.flink.util.TestLogger +import org.scalatest.junit.JUnitSuiteLike + +abstract class AcceptPFTestBase extends TestLogger with JUnitSuiteLike { + + private val env = ExecutionEnvironment.getExecutionEnvironment + + protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") + protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + + protected val groupedTuples = tuples.groupBy(_._1) + protected val groupedCaseObjects = caseObjects.groupBy(_.id) + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala new file mode 100644 index 0000000000000..ae807a940c1d8 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala @@ -0,0 +1,26 @@ +/* + * 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.api.scala.extensions.test.acceptPartialFunctions.data + +/** + * Simple case class to test the `acceptPartialFunctions` extension + * + * @param id A numerical identifier + * @param value A textual value + */ +private[acceptPartialFunctions] case class KeyValuePair(id: Int, value: String) From f1697783520af96897568429550413f041986655 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:39:21 +0100 Subject: [PATCH 25/39] [FLINK-1159] Add missing licenses, wrap long lines, add missing Scaladoc --- .../OnCoGroupDataSet.scala | 12 +++++- .../OnCrossDataSet.scala | 8 ++++ .../acceptPartialFunctions/OnDataSet.scala | 7 ++++ .../OnGroupedDataSet.scala | 7 ++++ .../OnHalfUnfinishedKeyPairOperation.scala | 37 +++++++++++++++++- .../OnJoinFunctionAssigner.scala | 13 ++++++- .../OnUnfinishedKeyPairOperation.scala | 38 ++++++++++++++++++- .../OnCoGroupDataSetTest.scala | 23 +++++++++-- .../OnCrossDataSetTest.scala | 23 +++++++++-- .../OnDataSetTest.scala | 7 ++-- .../OnGroupedDataSetTest.scala | 23 +++++++++-- ...OnHalfUnfinishedKeyPairOperationTest.scala | 23 +++++++++-- .../OnJoinFunctionAssignerTest.scala | 23 +++++++++-- .../OnUnfinishedKeyPairOperationTest.scala | 23 +++++++++-- .../extensions/base/AcceptPFTestBase.scala | 4 +- .../scala/extensions/data/KeyValuePair.scala | 4 +- 16 files changed, 243 insertions(+), 32 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala index 40fb729927c99..13db9c3b9fc92 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -22,7 +22,15 @@ import org.apache.flink.api.scala.{CoGroupDataSet, DataSet} import scala.reflect.ClassTag -class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](dataset: CoGroupDataSet[L, R]) { +/** + * Wraps a co-group data set, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped co-group data set + * @tparam L The type of the left data set items, for which the type information must be known + * @tparam R The type of the right data set items, for which the type information must be known + */ +class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](ds: CoGroupDataSet[L, R]) { /** * Co-groups the data sets using the function `fun` to project elements from both in @@ -33,7 +41,7 @@ class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](dataset: CoGroupD * @return A fully co-grouped data set of Os */ def projecting[O: TypeInformation: ClassTag](fun: (Stream[L], Stream[R]) => O): DataSet[O] = - dataset { + ds { (left, right) => fun(left.toStream, right.toStream) } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala index b0094f7c5e3d0..ec229f3b7fdaf 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala @@ -22,6 +22,14 @@ import org.apache.flink.api.scala.{CrossDataSet, DataSet} import scala.reflect.ClassTag +/** + * Wraps a cross data set, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped cross data set + * @tparam L The type of the left data set items, for which the type information must be known + * @tparam R The type of the right data set items, for which the type information must be known + */ class OnCrossDataSet[L: TypeInformation, R: TypeInformation](ds: CrossDataSet[L, R]) { /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala index eb3c6187a3917..d5b06d4f13575 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala @@ -22,6 +22,13 @@ import org.apache.flink.api.scala.{DataSet, GroupedDataSet} import scala.reflect.ClassTag +/** + * Wraps a data set, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped data set + * @tparam T The type of the data set items, for which the type information must be known + */ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala index 6252305a477a0..6ca9d9e430ad8 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala @@ -23,6 +23,13 @@ import org.apache.flink.api.scala.{DataSet, GroupedDataSet} import scala.reflect.ClassTag +/** + * Wraps a grouped data set, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped grouped data set + * @tparam T The type of the grouped data set items, for which the type information must be known + */ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala index 97bb4c7e99268..bfb8a57a91c6d 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala @@ -1,10 +1,45 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation +/** + * Wraps an half unfinished key pair operation, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped half unfinished key pair operation function assigner data set + * @tparam L The type of the left data set items, for which the type information must be known + * @tparam R The type of the right data set items, for which the type information must be known + * @tparam O The type of the output data set items, for which the type information must be known + */ -class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation](ds: HalfUnfinishedKeyPairOperation[L, R, O]) { +class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + ds: HalfUnfinishedKeyPairOperation[L, R, O]) { + /** + * Initiates a join or co-group operation, defining the second half of + * the where clause with an equality over the right data set items. + * + * @param fun The function that defines the equality of the where clause + * @tparam K The type of the key, for which type information must be known + * @return A data set of Os + */ def isEqualTo[K: TypeInformation](fun: R => K): O = ds.equalTo(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala index 53f28f15984de..bfe692ca52e1e 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala @@ -22,7 +22,16 @@ import org.apache.flink.api.scala.{DataSet, JoinFunctionAssigner} import scala.reflect.ClassTag -class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation](dataset: JoinFunctionAssigner[L, R]) { +/** + * Wraps a join function assigner, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped join function assigner data set + * @tparam L The type of the left data set items, for which the type information must be known + * @tparam R The type of the right data set items, for which the type information must be known + */ +class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation]( + ds: JoinFunctionAssigner[L, R]) { /** * Joins the data sets using the function `fun` to project elements from both in the @@ -33,6 +42,6 @@ class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation](dataset: Jo * @return A fully joined data set of Os */ def projecting[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O] = - dataset(fun) + ds(fun) } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala index d64a0754a18b3..3821431a272a1 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala @@ -1,10 +1,46 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{HalfUnfinishedKeyPairOperation, UnfinishedKeyPairOperation} -class OnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation](ds: UnfinishedKeyPairOperation[L, R, O]) { +/** + * Wraps an unfinished key pair operation, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param ds The wrapped unfinished key pair operation data set + * @tparam L The type of the left data set items, for which the type information must be known + * @tparam R The type of the right data set items, for which the type information must be known + * @tparam O The type of the output data set items, for which the type information must be known + */ +class OnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + ds: UnfinishedKeyPairOperation[L, R, O]) { + /** + * Initiates a join or co-group operation, defining the first half of + * the where clause with the items of the left data set that will be + * checked for equality with the ones provided by the second half. + * + * @param fun The function that defines the comparing item of the where clause + * @tparam K The type of the key, for which type information must be known + * @return A data set of Os + */ def whereClause[K: TypeInformation](fun: (L) => K): HalfUnfinishedKeyPairOperation[L, R, O] = ds.where(fun) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala index 8f9e3686442e0..da507acc816c2 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -1,13 +1,30 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CoGroupOperator import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test -class OnCoGroupDataSetTest extends AcceptPartialFunctionsTestBase { +class OnCoGroupDataSetTest extends AcceptPFTestBase { @Test def testCoGroupProjectingOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala index 1ed14fb43585e..650c9ab34b993 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala @@ -1,13 +1,30 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CrossOperator import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.junit.Test -class OnCrossDataSetTest extends AcceptPartialFunctionsTestBase { +class OnCrossDataSetTest extends AcceptPFTestBase { @Test def testCrossProjectingOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala index bcf63ca9a29dc..d4d62442e1576 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala @@ -15,17 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.junit.Test -class OnDataSetTest extends AcceptPartialFunctionsTestBase { +class OnDataSetTest extends AcceptPFTestBase { @Test def testMapWithOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala index 0e8aaa1c063a3..f35ec1aa048eb 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -1,14 +1,31 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.operators.{GroupCombineOperator, GroupReduceOperator, ReduceOperator} import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test -class OnGroupedDataSetTest extends AcceptPartialFunctionsTestBase { +class OnGroupedDataSetTest extends AcceptPFTestBase { @Test def testSortGroupWithOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala index 4ededf7a2bf9e..dca2208dd6132 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala @@ -1,14 +1,31 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CoGroupOperator import org.apache.flink.api.java.operators.JoinOperator.EquiJoin import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.junit.Test -class OnHalfUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { +class OnHalfUnfinishedKeyPairOperationTest extends AcceptPFTestBase { @Test def testInnerJoinIsEqualToOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala index d8c84190dbbbf..52e31ae86f998 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala @@ -1,13 +1,30 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.JoinOperator.EquiJoin import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.junit.Test -class OnJoinFunctionAssignerTest extends AcceptPartialFunctionsTestBase { +class OnJoinFunctionAssignerTest extends AcceptPFTestBase { @Test def testInnerJoinProjectingOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala index 24aa144f1e558..d23b107e75145 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala @@ -1,12 +1,29 @@ +/* + * 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.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.base.AcceptPartialFunctionsTestBase -import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions.data.KeyValuePair +import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.junit.Test -class OnUnfinishedKeyPairOperationTest extends AcceptPartialFunctionsTestBase { +class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { @Test def testInnerJoinWhereClauseOnTuple(): Unit = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala index ed5e2b8ea5514..58e9910386a8f 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.test.acceptPartialFunctions.base +package org.apache.flink.api.scala.extensions.base import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.apache.flink.util.TestLogger import org.scalatest.junit.JUnitSuiteLike -abstract class AcceptPFTestBase extends TestLogger with JUnitSuiteLike { +private[extensions] abstract class AcceptPFTestBase extends TestLogger with JUnitSuiteLike { private val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala index ae807a940c1d8..6d02393a8c769 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/data/KeyValuePair.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.scala.extensions.test.acceptPartialFunctions.data +package org.apache.flink.api.scala.extensions.data /** * Simple case class to test the `acceptPartialFunctions` extension @@ -23,4 +23,4 @@ package org.apache.flink.api.scala.extensions.test.acceptPartialFunctions.data * @param id A numerical identifier * @param value A textual value */ -private[acceptPartialFunctions] case class KeyValuePair(id: Int, value: String) +private [extensions] case class KeyValuePair(id: Int, value: String) From 697f645791c5d402b31b6b6d9189be169e07ef3c Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:43:44 +0100 Subject: [PATCH 26/39] [FLINK-1159] Improve Scaladoc for package --- .../apache/flink/api/scala/extensions/package.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index ae9f24daa6c9d..9b0b11ba4a13e 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -29,7 +29,9 @@ import scala.reflect.ClassTag * functions as well. This enables the possibility to directly apply pattern matching * to decompose inputs such as tuples, case classes and collections. * - * e.g. + * The following is a small example that showcases how this extensions would work on + * a Flink data set: + * * {{{ * object Main { * import org.apache.flink.api.scala.extensions._ @@ -52,6 +54,12 @@ import scala.reflect.ClassTag * } * }}} * + * The extension consists of several implicit conversions over all the data set representations + * that could gain from this feature. To use this set of extensions methods the user has to + * explicitly opt-in by importing `org.apache.flink.api.scala.extensions.acceptPartialFunctions`. + * + * For more information and usage examples please consult the Apache Flink official documentation. + * */ package object extensions { From c54e611f3dc57da89bec321cf6ea40a182013711 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:48:09 +0100 Subject: [PATCH 27/39] [FLINK-1159] Prevent long line issue --- .../OnGroupedDataSetTest.scala | 2 +- .../OnUnfinishedKeyPairOperationTest.scala | 27 ++++++++++++------- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala index f35ec1aa048eb..82b6bf7af960d 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.operators.Order -import org.apache.flink.api.java.operators.{GroupCombineOperator, GroupReduceOperator, ReduceOperator} +import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.api.scala.extensions.data.KeyValuePair diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala index d23b107e75145..b454699341652 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala @@ -42,7 +42,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for inner join on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for inner join on case objects " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -52,7 +53,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for right outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for right outer join on tuples " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -62,7 +64,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for right outer join on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for right outer join on case objects " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -72,7 +75,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for left outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for left outer join on tuples " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -82,7 +86,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for left outer join on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for left outer join on case objects " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -92,7 +97,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for full outer join on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for full outer join on tuples " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -102,7 +108,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for full outer join on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for full outer join on case objects " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -112,7 +119,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case (id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for co-group on tuples should produce a HalfUnfinishedKeyPairOperation") + "whereClause for co-group on tuples " + + "should produce a HalfUnfinishedKeyPairOperation") } @Test @@ -122,7 +130,8 @@ class OnUnfinishedKeyPairOperationTest extends AcceptPFTestBase { case KeyValuePair(id, _) => id } assert(test.isInstanceOf[HalfUnfinishedKeyPairOperation[_, _, _]], - "whereClause for co-group on case objects should produce a HalfUnfinishedKeyPairOperation") + "whereClause for co-group on case objects " + + "should produce a HalfUnfinishedKeyPairOperation") } } From fd4a649d6bbd6042b1c8c266c44b9342e8878783 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:52:57 +0100 Subject: [PATCH 28/39] [FLINK-1159] Improve DataStream ext. Scaladoc, provide return types --- .../api/scala/extensions/package.scala | 201 ++++-------------- 1 file changed, 43 insertions(+), 158 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index eca79ea946f23..c11a137fcb483 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -21,182 +21,67 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions._ import org.apache.flink.streaming.api.windowing.windows.Window +/** + * acceptPartialFunctions extends the original DataStream with methods with unique names + * that delegate to core higher-order functions (e.g. `map`) so that we can work around + * the fact that overloaded methods taking functions as parameters can't accept partial + * functions as well. This enables the possibility to directly apply pattern matching + * to decompose inputs such as tuples, case classes and collections. + * + * The following is a small example that showcases how this extensions would work on + * a Flink data stream: + * + * {{{ + * object Main { + * import org.apache.flink.streaming.api.scala.extensions._ + * case class Point(x: Double, y: Double) + * def main(args: Array[String]): Unit = { + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) + * ds.filterWith { + * case Point(x, _) => x > 1 + * }.reduceWith { + * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) + * }.mapWith { + * case Point(x, y) => (x, y) + * }.flatMapWith { + * case (x, y) => Seq('x' -> x, 'y' -> y) + * }.keyingBy { + * case (id, value) => id + * } + * } + * } + * }}} + * + * The extension consists of several implicit conversions over all the data stream representations + * that could gain from this feature. To use this set of extensions methods the user has to + * explicitly opt-in by importing + * `org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions`. + * + * For more information and usage examples please consult the Apache Flink official documentation. + * + */ package object extensions { - /** - * acceptPartialFunctions extends the original DataStream with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = StreamExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.keyingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnDataStream[T: TypeInformation](ds: DataStream[T]): OnDataStream[T] = new OnDataStream[T](ds) - /** - * acceptPartialFunctions extends the original DataStream with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = StreamExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.keyingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnKeyedStream[T: TypeInformation, K](ds: KeyedStream[T, K]): OnKeyedStream[T, K] = new OnKeyedStream[T, K](ds) - /** - * acceptPartialFunctions extends the original DataStream with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = StreamExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.keyingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnJoinedStream [L: TypeInformation, R: TypeInformation, K, W <: Window]( - ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) = + ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]): OnJoinedStream[L, R, K, W] = new OnJoinedStream[L, R, K, W](ds) - /** - * acceptPartialFunctions extends the original DataStream with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = StreamExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.keyingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( - ds: ConnectedStreams[IN1, IN2]) = + ds: ConnectedStreams[IN1, IN2]): OnConnectedStream[IN1, IN2] = new OnConnectedStream[IN1, IN2](ds) - /** - * acceptPartialFunctions extends the original DataStream with methods with unique names - * that delegate to core higher-order functions (e.g. `map`) so that we can work around - * the fact that overloaded methods taking functions as parameters can't accept partial - * functions as well. This enables the possibility to directly apply pattern matching - * to decompose inputs such as tuples, case classes and collections. - * - * e.g. - * {{{ - * object Main { - * import org.apache.flink.api.scala.extensions._ - * case class Point(x: Double, y: Double) - * def main(args: Array[String]): Unit = { - * val env = StreamExecutionEnvironment.getExecutionEnvironment - * val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6)) - * ds.filterWith { - * case Point(x, _) => x > 1 - * }.reduceWith { - * case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2) - * }.mapWith { - * case Point(x, y) => (x, y) - * }.flatMapWith { - * case (x, y) => Seq('x' -> x, 'y' -> y) - * }.keyingBy { - * case (id, value) => id - * } - * } - * } - * }}} - * - */ implicit def acceptPartialFunctionsOnWindowedStream[T, K, W <: Window]( - ds: WindowedStream[T, K, W]) = + ds: WindowedStream[T, K, W]): OnWindowedStream[T, K, W] = new OnWindowedStream[T, K, W](ds) } From fc20f3c28073578daecd519377614652eba9a252 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 25 Mar 2016 20:58:12 +0100 Subject: [PATCH 29/39] [FLINK-1159] Move implementations to a different package, make the extension importable under one name --- .../{ => impl}/acceptPartialFunctions/OnConnectedStream.scala | 4 ++-- .../{ => impl}/acceptPartialFunctions/OnDataStream.scala | 4 ++-- .../{ => impl}/acceptPartialFunctions/OnJoinedStream.scala | 2 +- .../{ => impl}/acceptPartialFunctions/OnKeyedStream.scala | 2 +- .../{ => impl}/acceptPartialFunctions/OnWindowedStream.scala | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) rename flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnConnectedStream.scala (94%) rename flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnDataStream.scala (93%) rename flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnJoinedStream.scala (94%) rename flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnKeyedStream.scala (95%) rename flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnWindowedStream.scala (97%) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala similarity index 94% rename from flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala rename to flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala index cdd0ee2d7f1ee..fe4b9a38570ba 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.scala.{DataStream, ConnectedStreams} +import org.apache.flink.streaming.api.scala.{ConnectedStreams, DataStream} class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( ds: ConnectedStreams[IN1, IN2]) { diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala similarity index 93% rename from flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala rename to flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala index 3ba11babacf46..c64e7b2b2999f 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.scala.{KeyedStream, DataStream} +import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} class OnDataStream[T: TypeInformation](stream: DataStream[T]) { diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala similarity index 94% rename from flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala rename to flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index 380684810b1c9..07f4c62554714 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala similarity index 95% rename from flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala rename to flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala index ceeb8fb5a7add..0c7663d29b5da 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala similarity index 97% rename from flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala rename to flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala index 417b9ac6dda15..2411dfeee60e1 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, WindowedStream} From 2b7bd2a48f1d5cf09aaf14a695dd9bb4771725fd Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sun, 27 Mar 2016 18:50:09 +0200 Subject: [PATCH 30/39] [FLINK-1159] Add Scaldoc to AcceptPFTestBase --- .../flink/api/scala/extensions/base/AcceptPFTestBase.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala index 58e9910386a8f..c2e13fead84f0 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/base/AcceptPFTestBase.scala @@ -22,6 +22,9 @@ import org.apache.flink.api.scala.extensions.data.KeyValuePair import org.apache.flink.util.TestLogger import org.scalatest.junit.JUnitSuiteLike +/** + * Common facilities to test the `acceptPartialFunctions` extension + */ private[extensions] abstract class AcceptPFTestBase extends TestLogger with JUnitSuiteLike { private val env = ExecutionEnvironment.getExecutionEnvironment From d2dffa0aed4f012ba2baff1aa68c614e0e0b95d7 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sun, 27 Mar 2016 18:51:22 +0200 Subject: [PATCH 31/39] [FLINK-1159] Overload streaming extension method name to import it in one go --- .../streaming/api/scala/extensions/package.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index c11a137fcb483..d18cd9ac7c3a6 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -18,7 +18,7 @@ package org.apache.flink.streaming.api.scala import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions._ +import org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions._ import org.apache.flink.streaming.api.windowing.windows.Window /** @@ -63,24 +63,23 @@ import org.apache.flink.streaming.api.windowing.windows.Window */ package object extensions { - implicit def acceptPartialFunctionsOnDataStream[T: TypeInformation](ds: DataStream[T]): + implicit def acceptPartialFunctions[T: TypeInformation](ds: DataStream[T]): OnDataStream[T] = new OnDataStream[T](ds) - implicit def acceptPartialFunctionsOnKeyedStream[T: TypeInformation, K](ds: KeyedStream[T, K]): + implicit def acceptPartialFunctions[T: TypeInformation, K](ds: KeyedStream[T, K]): OnKeyedStream[T, K] = new OnKeyedStream[T, K](ds) - implicit def acceptPartialFunctionsOnJoinedStream - [L: TypeInformation, R: TypeInformation, K, W <: Window]( + implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, K, W <: Window]( ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]): OnJoinedStream[L, R, K, W] = new OnJoinedStream[L, R, K, W](ds) - implicit def acceptPartialFunctionsOnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( + implicit def acceptPartialFunctions[IN1: TypeInformation, IN2: TypeInformation]( ds: ConnectedStreams[IN1, IN2]): OnConnectedStream[IN1, IN2] = new OnConnectedStream[IN1, IN2](ds) - implicit def acceptPartialFunctionsOnWindowedStream[T, K, W <: Window]( + implicit def acceptPartialFunctions[T, K, W <: Window]( ds: WindowedStream[T, K, W]): OnWindowedStream[T, K, W] = new OnWindowedStream[T, K, W](ds) From 6f5f437f4d468f77fa0516985384026eb82aeb91 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Sun, 27 Mar 2016 18:51:56 +0200 Subject: [PATCH 32/39] [FLINK-1159] Add first tests for DataStream and KeyedDataStream --- .../OnDataStreamTest.scala | 110 ++++++++++++++++++ .../OnKeyedDataStreamTest.scala | 69 +++++++++++ .../extensions/base/AcceptPFTestBase.scala | 39 +++++++ .../scala/extensions/data/KeyValuePair.scala | 26 +++++ 4 files changed, 244 insertions(+) create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/data/KeyValuePair.scala diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala new file mode 100644 index 0000000000000..a415fdb790619 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala @@ -0,0 +1,110 @@ +/* + * 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.extensions.impl.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator +import org.apache.flink.streaming.api.datastream.KeyedStream +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.junit.Test + +class OnDataStreamTest extends AcceptPFTestBase { + + @Test + def testMapWithOnTuple(): Unit = { + val test = + tuples.mapWith { + case (id, value) => s"$id $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "mapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testMapWithOnCaseClass(): Unit = { + val test = + caseObjects.mapWith { + case KeyValuePair(id, value) => s"$id $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "mapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFlatMapWithOnTuple(): Unit = { + val test = + tuples.flatMapWith { + case (id, value) => List(id.toString, value) + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFlatMapWithOnCaseClass(): Unit = { + val test = + caseObjects.flatMapWith { + case KeyValuePair(id, value) => List(id.toString, value) + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFilterWithOnTuple(): Unit = { + val test = + tuples.filterWith { + case (id, value) => id == 1 + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "filterWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFilterWithOnCaseClass(): Unit = { + val test = + caseObjects.filterWith { + case KeyValuePair(id, value) => id == 1 + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "filterWith should produce a SingleOutputStreamOperator") + } + + @Test + def testKeyingByOnTuple(): Unit = { + val test = + tuples.keyingBy { + case (id, _) => id + } + assert(test.javaStream.isInstanceOf[KeyedStream[_, _]], + "keyingBy should produce a KeyedStream") + } + + @Test + def testKeyingByOnCaseClass(): Unit = { + val test = + caseObjects.keyingBy { + case KeyValuePair(id, _) => id + } + assert(test.javaStream.isInstanceOf[KeyedStream[_, _]], + "keyingBy should produce a KeyedStream") + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala new file mode 100644 index 0000000000000..876a5f3f138ba --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala @@ -0,0 +1,69 @@ +/* + * 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.extensions.impl.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.datastream.{KeyedStream, SingleOutputStreamOperator} +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.junit.Test + +class OnKeyedDataStreamTest extends AcceptPFTestBase { + + @Test + def testReduceWithOnTuple(): Unit = { + val test = + keyedTuples.reduceWith { + case ((_, v1), (_, v2)) => 0 -> s"$v1 $v2" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "reduceWith should produce a SingleOutputStreamOperator") + } + + @Test + def testReduceWithOnCaseClass(): Unit = { + val test = + keyedCaseObjects.reduceWith { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => KeyValuePair(0, s"$v1 $v2") + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "reduceWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFoldWithOnTuple(): Unit = { + val test = + keyedTuples.foldWith("") { + case (folding, (_, value)) => s"$folding $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFoldWithOnCaseClass(): Unit = { + val test = + keyedCaseObjects.foldWith("") { + case (folding, KeyValuePair(_, value)) => s"$folding $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala new file mode 100644 index 0000000000000..f8f9b19e7a797 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.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.extensions.base + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.apache.flink.util.TestLogger +import org.scalatest.junit.JUnitSuiteLike + +/** + * Common facilities to test the `acceptPartialFunctions` extension + */ +private[extensions] abstract class AcceptPFTestBase extends TestLogger with JUnitSuiteLike { + + private val env = StreamExecutionEnvironment.getExecutionEnvironment + + protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") + protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + + protected val keyedTuples = tuples.keyBy(_._1) + protected val keyedCaseObjects = caseObjects.keyBy(_.id) + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/data/KeyValuePair.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/data/KeyValuePair.scala new file mode 100644 index 0000000000000..fad818938dcdd --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/data/KeyValuePair.scala @@ -0,0 +1,26 @@ +/* + * 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.extensions.data + +/** + * Simple case class to test the `acceptPartialFunctions` extension + * + * @param id A numerical identifier + * @param value A textual value + */ +private [extensions] case class KeyValuePair(id: Int, value: String) From 0caea8f3b76f0d60a11fc791d9436af3d7920705 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Thu, 31 Mar 2016 15:22:56 +0200 Subject: [PATCH 33/39] [FLINK-1159] Update documentation --- docs/apis/scala_api_extensions.md | 43 +++++++++++++++++++++---------- 1 file changed, 30 insertions(+), 13 deletions(-) diff --git a/docs/apis/scala_api_extensions.md b/docs/apis/scala_api_extensions.md index 4e91a477bbc2b..e3268bf9d0b41 100644 --- a/docs/apis/scala_api_extensions.md +++ b/docs/apis/scala_api_extensions.md @@ -94,7 +94,7 @@ data.mapWith { {% highlight scala %} data.mapPartitionWith { - case head +: _ => head + case head #:: _ => head } {% endhighlight %} @@ -138,7 +138,7 @@ data.reduceWith { {% highlight scala %} data.reduceGroupWith { - case id +: value +: _ => id -> value + case id #:: value #:: _ => id -> value } {% endhighlight %} @@ -171,7 +171,7 @@ grouped.sortGroupWith(Order.ASCENDING) { {% highlight scala %} grouped.combineGroupWith { - case header +: amounts => amounts.sum + case header #:: amounts => amounts.sum } {% endhighlight %} @@ -180,9 +180,12 @@ grouped.combineGroupWith { apply (JoinDataSet, CrossDataSet) {% highlight scala %} -data1.join(data2).where(0).equalTo(1).projecting { - case ((pk, tx), (products, fk)) => tx -> products -} +data1.join(data2). + whereClause(case (pk, _) => pk). + isEqualTo(case (_, fk) => fk). + projecting { + case ((pk, tx), (products, fk)) => tx -> products + } data1.cross(data2).projecting { case ((a, _), (_, b) => a -> b @@ -195,8 +198,12 @@ data1.cross(data2).projecting { apply (CoGroupDataSet) {% highlight scala %} -data1.coGroup(data2).where(0).equalTo(1).projecting { - case (head1 +: _, head2 +: _) => head1 -> head2 +data1.coGroup(data2). + whereClause(case (pk, _) => pk). + isEqualTo(case (_, fk) => fk). + projecting { + case (head1 #:: _, head2 #:: _) => head1 -> head2 + } } {% endhighlight %} @@ -234,7 +241,7 @@ data.mapWith { {% highlight scala %} data.mapPartitionWith { - case head +: _ => head + case head #:: _ => head } {% endhighlight %} @@ -347,9 +354,12 @@ data.applyWith(0)( apply (JoinedDataStream) {% highlight scala %} -data1.join(data2).where(0).equalTo(1).projecting { - case ((pk, tx), (products, fk)) => tx -> products -} +data1.join(data2). + whereClause(case (pk, _) => pk). + isEqualTo(case (_, fk) => fk). + projecting { + case ((pk, tx), (products, fk)) => tx -> products + } {% endhighlight %} @@ -367,7 +377,14 @@ To use this extension exclusively, you can add the following `import`: import org.apache.flink.api.scala.extensions.acceptPartialFunctions {% endhighlight %} -The following snippet shows a minimal example of how to use these extension methods together: +for the DataSet extensions and + +{% highlight scala %} +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +{% endhighlight %} + +The following snippet shows a minimal example of how to use these extension +methods together (with the DataSet API): {% highlight scala %} object Main { From 5627e63f473a13b3815a7cfc6296e381ea38c99a Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 1 Apr 2016 10:23:25 +0200 Subject: [PATCH 34/39] [FLINK-1159] Add tests for connected, joined and windowed streams --- .../streaming/api/scala/JoinedStreams.scala | 2 +- .../OnJoinedStream.scala | 2 +- .../OnConnectedDataStreamTest.scala | 102 ++++++++++++++++++ .../OnJoinedDataStreamTest.scala | 66 ++++++++++++ .../OnWindowedDataStreamTest.scala | 97 +++++++++++++++++ .../extensions/base/AcceptPFTestBase.scala | 19 +++- 6 files changed, 284 insertions(+), 4 deletions(-) create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala index 4d09dae860759..93b5cc885b4ee 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala @@ -73,7 +73,7 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { * A join operation that has a [[KeySelector]] defined for the first input. * * You need to specify a [[KeySelector]] for the second input using [[equalTo()]] - * before you can proceeed with specifying a [[WindowAssigner]] using [[EqualTo.window()]]. + * before you can proceed with specifying a [[WindowAssigner]] using [[EqualTo.window()]]. * * @tparam KEY Type of the key. This must be the same for both inputs */ diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index 07f4c62554714..a2b4fdd5af642 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} import org.apache.flink.streaming.api.windowing.windows.Window -class OnJoinedStream[L: TypeInformation, R: TypeInformation, K, W <: Window]( +class OnJoinedStream[L, R, K, W <: Window]( stream: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) { /** diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala new file mode 100644 index 0000000000000..60839ce48c690 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala @@ -0,0 +1,102 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator +import org.apache.flink.streaming.api.scala.ConnectedStreams +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.junit.Test + +class OnConnectedDataStreamTest extends AcceptPFTestBase { + + @Test + def testMapWithOnTuple(): Unit = { + val test = + tuples.connect(tuples).mapWith({ + case (id, value) => s"$id $value" + }, { + case (id, value) => s"$id $value" + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "mapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testMapWithOnCaseClass(): Unit = { + val test = + caseObjects.connect(caseObjects).mapWith({ + case KeyValuePair(id, value) => s"$id $value" + }, { + case KeyValuePair(id, value) => s"$id $value" + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "mapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFlatMapWithOnTuple(): Unit = { + val test = + tuples.connect(tuples).flatMapWith({ + case (id, value) => List(id.toString, value) + }, { + case (id, value) => List(id.toString, value) + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFlatMapWithOnCaseClass(): Unit = { + val test = + caseObjects.connect(caseObjects).flatMapWith({ + case KeyValuePair(id, value) => List(id.toString, value) + }, { + case KeyValuePair(id, value) => List(id.toString, value) + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "flatMapWith should produce a SingleOutputStreamOperator") + } + + @Test + def testKeyingByOnTuple(): Unit = { + val test = + tuples.connect(tuples).keyingBy({ + case (id, _) => id + }, { + case (id, _) => id + }) + assert(test.isInstanceOf[ConnectedStreams[_, _]], + "keyingBy should produce a ConnectedStreams") + } + + @Test + def testKeyingByOnCaseClass(): Unit = { + val test = + caseObjects.connect(caseObjects).keyingBy({ + case KeyValuePair(id, _) => id + }, { + case KeyValuePair(id, _) => id + }) + assert(test.isInstanceOf[ConnectedStreams[_, _]], + "keyingBy should produce a ConnectedStreams") + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala new file mode 100644 index 0000000000000..6307de3d423c2 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala @@ -0,0 +1,66 @@ +/* + * 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.extensions.impl.acceptPartialFunctions + +import java.util.concurrent.TimeUnit + +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator +import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows +import org.apache.flink.streaming.api.windowing.time.Time +import org.junit.Test + +class OnJoinedDataStreamTest extends AcceptPFTestBase { + + @Test + def testProjectingOnTuple(): Unit = { + val test = + tuples.join(tuples). + where { + case (id, _) => id + }.equalTo { + case (id, _) => id + }.window { + TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)) + }.projecting { + case ((_, v1), (_, v2)) => s"$v1 $v2" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "projecting should produce a SingleOutputStreamOperator") + } + + @Test + def testProjectingOnCaseClass(): Unit = { + val test = + tuples.join(tuples). + where { + case (id, _) => id + }.equalTo { + case (id, _) => id + }.window { + TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)) + }.projecting { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => s"$v1 $v2" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "projecting should produce a SingleOutputStreamOperator") + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala new file mode 100644 index 0000000000000..c0434500f1620 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala @@ -0,0 +1,97 @@ +/* + * 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.extensions.acceptPartialFunctions + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator +import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase +import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair +import org.junit.Test + +class OnWindowedDataStreamTest extends AcceptPFTestBase { + + @Test + def testReduceWithOnTuple(): Unit = { + val test = + windowedTuples.reduceWith { + case ((_, v1), (_, v2)) => 0 -> s"$v1 $v2" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "reduceWith should produce a SingleOutputStreamOperator") + } + + @Test + def testReduceWithOnCaseClass(): Unit = { + val test = + windowedCaseObjects.reduceWith { + case (KeyValuePair(_, v1), KeyValuePair(_, v2)) => KeyValuePair(0, s"$v1 $v2") + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "reduceWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFoldWithOnTuple(): Unit = { + val test = + windowedTuples.foldWith("") { + case (folding, (_, value)) => s"$folding $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "foldWith should produce a SingleOutputStreamOperator") + } + + @Test + def testFoldWithOnCaseClass(): Unit = { + val test = + windowedCaseObjects.foldWith("") { + case (folding, KeyValuePair(_, value)) => s"$folding $value" + } + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "foldWith should produce a SingleOutputStreamOperator") + } + + @Test + def testApplyWithOnTuple(): Unit = { + val test = + windowedTuples.applyWith("")( + foldFunction = { + case (folding, (_, value)) => s"$folding $value" + }, + windowFunction = { + case (n, w, head #:: neck #:: _) => Seq(n.toString, w.maxTimestamp().toString, head, neck) + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "applyWith should produce a SingleOutputStreamOperator") + } + + @Test + def testApplyWithOnCaseClass(): Unit = { + val test = + windowedCaseObjects.applyWith("")( + foldFunction = { + case (folding, KeyValuePair(_, value)) => s"$folding $value" + }, + windowFunction = { + case (n, w, head #:: neck #:: _) => Seq(n.toString, w.maxTimestamp().toString, head, neck) + }) + assert(test.javaStream.isInstanceOf[SingleOutputStreamOperator[_]], + "applyWith should produce a SingleOutputStreamOperator") + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala index f8f9b19e7a797..86176e66e8ed0 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/base/AcceptPFTestBase.scala @@ -30,10 +30,25 @@ private[extensions] abstract class AcceptPFTestBase extends TestLogger with JUni private val env = StreamExecutionEnvironment.getExecutionEnvironment - protected val tuples = env.fromElements(1 -> "hello", 2 -> "world") - protected val caseObjects = env.fromElements(KeyValuePair(1, "hello"), KeyValuePair(2, "world")) + protected val tuples = env.fromElements( + 1 -> "hello", + 2 -> "world", + 3 -> "foo", + 4 -> "bar", + 5 -> "baz", + 6 -> "quux") + protected val caseObjects = env.fromElements( + KeyValuePair(1, "hello"), + KeyValuePair(2, "world"), + KeyValuePair(3, "foo"), + KeyValuePair(4, "bar"), + KeyValuePair(5, "baz"), + KeyValuePair(6, "quux")) protected val keyedTuples = tuples.keyBy(_._1) protected val keyedCaseObjects = caseObjects.keyBy(_.id) + protected val windowedTuples = keyedTuples.countWindow(2) + protected val windowedCaseObjects = keyedCaseObjects.countWindow(2) + } From 38a00847c396b515aa1f42e6cedb19d71d351bcb Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Fri, 1 Apr 2016 11:07:56 +0200 Subject: [PATCH 35/39] [FLINK-1159] Fix several compilation issues --- .../impl/acceptPartialFunctions/OnJoinedStream.scala | 2 +- .../acceptPartialFunctions/OnConnectedDataStreamTest.scala | 2 +- .../acceptPartialFunctions/OnJoinedDataStreamTest.scala | 7 ++++--- .../acceptPartialFunctions/OnWindowedDataStreamTest.scala | 2 +- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index a2b4fdd5af642..07f4c62554714 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} import org.apache.flink.streaming.api.windowing.windows.Window -class OnJoinedStream[L, R, K, W <: Window]( +class OnJoinedStream[L: TypeInformation, R: TypeInformation, K, W <: Window]( stream: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) { /** diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala index 60839ce48c690..f5bb89aeb9c1e 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala index 6307de3d423c2..823f61ebbfb50 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFuncti import java.util.concurrent.TimeUnit +import org.apache.flink.streaming.api.scala._ import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase @@ -49,11 +50,11 @@ class OnJoinedDataStreamTest extends AcceptPFTestBase { @Test def testProjectingOnCaseClass(): Unit = { val test = - tuples.join(tuples). + caseObjects.join(caseObjects). where { - case (id, _) => id + case KeyValuePair(id, _) => id }.equalTo { - case (id, _) => id + case KeyValuePair(id, _) => id }.window { TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)) }.projecting { diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala index c0434500f1620..4fa9f5a7ba0d7 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions +package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator From 6b1bc1cdf3b38253654f1ada09c0bbe709d08cc7 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Mon, 4 Apr 2016 10:11:13 +0200 Subject: [PATCH 36/39] [FLINK-1159] Add @PublicEvolving to APIs, keep package and directory structure in sync --- .../impl/acceptPartialFunctions/OnCoGroupDataSet.scala | 2 ++ .../impl/acceptPartialFunctions/OnCrossDataSet.scala | 2 ++ .../impl/acceptPartialFunctions/OnDataSet.scala | 8 ++++++++ .../impl/acceptPartialFunctions/OnGroupedDataSet.scala | 5 +++++ .../OnHalfUnfinishedKeyPairOperation.scala | 3 ++- .../acceptPartialFunctions/OnJoinFunctionAssigner.scala | 2 ++ .../OnUnfinishedKeyPairOperation.scala | 2 ++ .../org/apache/flink/api/scala/extensions/package.scala | 8 ++++++++ .../acceptPartialFunctions/OnCoGroupDataSetTest.scala | 2 +- .../acceptPartialFunctions/OnCrossDataSetTest.scala | 0 .../{ => impl}/acceptPartialFunctions/OnDataSetTest.scala | 0 .../acceptPartialFunctions/OnGroupedDataSetTest.scala | 2 +- .../OnHalfUnfinishedKeyPairOperationTest.scala | 0 .../OnJoinFunctionAssignerTest.scala | 0 .../OnUnfinishedKeyPairOperationTest.scala | 0 .../impl/acceptPartialFunctions/OnConnectedStream.scala | 4 ++++ .../impl/acceptPartialFunctions/OnDataStream.scala | 5 +++++ .../impl/acceptPartialFunctions/OnJoinedStream.scala | 2 ++ .../impl/acceptPartialFunctions/OnKeyedStream.scala | 3 +++ .../impl/acceptPartialFunctions/OnWindowedStream.scala | 4 ++++ .../flink/streaming/api/scala/extensions/package.scala | 6 ++++++ .../OnConnectedDataStreamTest.scala | 0 .../acceptPartialFunctions/OnDataStreamTest.scala | 3 +-- .../acceptPartialFunctions/OnJoinedDataStreamTest.scala | 2 +- .../acceptPartialFunctions/OnKeyedDataStreamTest.scala | 2 +- .../acceptPartialFunctions/OnWindowedDataStreamTest.scala | 0 26 files changed, 60 insertions(+), 7 deletions(-) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnCoGroupDataSetTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnCrossDataSetTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnDataSetTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnGroupedDataSetTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala (100%) rename flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala (100%) rename flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnConnectedDataStreamTest.scala (100%) rename flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnDataStreamTest.scala (96%) rename flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnJoinedDataStreamTest.scala (100%) rename flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnKeyedDataStreamTest.scala (96%) rename flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/{ => impl}/acceptPartialFunctions/OnWindowedDataStreamTest.scala (100%) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala index 13db9c3b9fc92..485e3525b1d07 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{CoGroupDataSet, DataSet} @@ -40,6 +41,7 @@ class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](ds: CoGroupDataSe * @tparam O The return type of the projection, for which type information must be known * @return A fully co-grouped data set of Os */ + @PublicEvolving def projecting[O: TypeInformation: ClassTag](fun: (Stream[L], Stream[R]) => O): DataSet[O] = ds { (left, right) => diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala index ec229f3b7fdaf..6a6ad76deb825 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{CrossDataSet, DataSet} @@ -40,6 +41,7 @@ class OnCrossDataSet[L: TypeInformation, R: TypeInformation](ds: CrossDataSet[L, * @tparam O The return type of the projection, for which type information must be known * @return A data set of Os */ + @PublicEvolving def projecting[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O] = ds(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala index d5b06d4f13575..96c8b6ba85fbf 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{DataSet, GroupedDataSet} @@ -38,6 +39,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of R */ + @PublicEvolving def mapWith[R: TypeInformation: ClassTag](fun: T => R): DataSet[R] = ds.map(fun) @@ -48,6 +50,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of R */ + @PublicEvolving def mapPartitionWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.mapPartition { (it, out) => @@ -62,6 +65,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of R */ + @PublicEvolving def flatMapWith[R: TypeInformation: ClassTag](fun: T => TraversableOnce[R]): DataSet[R] = ds.flatMap(fun) @@ -72,6 +76,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @param fun The predicate to be tested on each item * @return A dataset of R */ + @PublicEvolving def filterWith(fun: T => Boolean): DataSet[T] = ds.filter(fun) @@ -82,6 +87,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned collection * @return A data set of Rs */ + @PublicEvolving def reduceWith[R: TypeInformation](fun: (T, T) => T): DataSet[T] = ds.reduce(fun) @@ -92,6 +98,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam R The type of the items in the returned data set * @return A dataset of Rs */ + @PublicEvolving def reduceGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.reduceGroup { (it, out) => @@ -105,6 +112,7 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * @tparam K The return type of the grouping function, for which type information must be known * @return A grouped data set of Ts */ + @PublicEvolving def groupingBy[K: TypeInformation](fun: T => K): GroupedDataSet[T] = ds.groupBy(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala index 6ca9d9e430ad8..71a7647fc0f88 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.operators.Order import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{DataSet, GroupedDataSet} @@ -40,6 +41,7 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @tparam K The key type * @return A data set sorted group-wise */ + @PublicEvolving def sortGroupWith[K: TypeInformation](order: Order)(fun: T => K): GroupedDataSet[T] = ds.sortGroup(fun, order) @@ -49,6 +51,7 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @param fun The reducing function * @return A reduced data set of Ts */ + @PublicEvolving def reduceWith(fun: (T, T) => T): DataSet[T] = ds.reduce(fun) @@ -59,6 +62,7 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @tparam R The type of the items in the resulting data set * @return A data set of Rs reduced group-wise */ + @PublicEvolving def reduceGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.reduceGroup { (it, out) => @@ -73,6 +77,7 @@ class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { * @tparam R The type of the items in the resulting data set * @return A data set of Rs reduced group-wise */ + @PublicEvolving def combineGroupWith[R: TypeInformation: ClassTag](fun: Stream[T] => R): DataSet[R] = ds.combineGroup { (it, out) => diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala index bfb8a57a91c6d..86eb2b4eaca0a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation /** @@ -28,7 +29,6 @@ import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation * @tparam R The type of the right data set items, for which the type information must be known * @tparam O The type of the output data set items, for which the type information must be known */ - class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( ds: HalfUnfinishedKeyPairOperation[L, R, O]) { @@ -40,6 +40,7 @@ class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O * @tparam K The type of the key, for which type information must be known * @return A data set of Os */ + @PublicEvolving def isEqualTo[K: TypeInformation](fun: R => K): O = ds.equalTo(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala index bfe692ca52e1e..ce24e92b7aa79 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{DataSet, JoinFunctionAssigner} @@ -41,6 +42,7 @@ class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation]( * @tparam O The return type of the projection, for which type information must be known * @return A fully joined data set of Os */ + @PublicEvolving def projecting[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O] = ds(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala index 3821431a272a1..69bad019f446d 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.{HalfUnfinishedKeyPairOperation, UnfinishedKeyPairOperation} @@ -41,6 +42,7 @@ class OnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: Ty * @tparam K The type of the key, for which type information must be known * @return A data set of Os */ + @PublicEvolving def whereClause[K: TypeInformation](fun: (L) => K): HalfUnfinishedKeyPairOperation[L, R, O] = ds.where(fun) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index 9b0b11ba4a13e..56f16f22e0bee 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions._ @@ -63,30 +64,37 @@ import scala.reflect.ClassTag */ package object extensions { + @PublicEvolving implicit def acceptPartialFunctions[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = new OnDataSet[T](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: JoinFunctionAssigner[L, R]): OnJoinFunctionAssigner[L, R] = new OnJoinFunctionAssigner[L, R](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = new OnCrossDataSet[L, R](ds) + @PublicEvolving implicit def acceptPartialFunctions[T: TypeInformation: ClassTag]( ds: GroupedDataSet[T]): OnGroupedDataSet[T] = new OnGroupedDataSet[T](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = new OnCoGroupDataSet[L, R](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( ds: HalfUnfinishedKeyPairOperation[L, R, O]): OnHalfUnfinishedKeyPairOperation[L, R, O] = new OnHalfUnfinishedKeyPairOperation[L, R, O](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( ds: UnfinishedKeyPairOperation[L, R, O]): OnUnfinishedKeyPairOperation[L, R, O] = new OnUnfinishedKeyPairOperation[L, R, O](ds) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSetTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSetTest.scala index da507acc816c2..a20f977557b93 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCoGroupDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSetTest.scala @@ -19,9 +19,9 @@ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.java.operators.CoGroupOperator import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.api.scala.extensions.data.KeyValuePair -import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test class OnCoGroupDataSetTest extends AcceptPFTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSetTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnCrossDataSetTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSetTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSetTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnDataSetTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSetTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSetTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSetTest.scala index 82b6bf7af960d..898c4b01c9d82 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnGroupedDataSetTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSetTest.scala @@ -20,9 +20,9 @@ package org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.apache.flink.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.api.scala.extensions.data.KeyValuePair -import org.apache.flink.api.scala.extensions.acceptPartialFunctions import org.junit.Test class OnGroupedDataSetTest extends AcceptPFTestBase { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperationTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssignerTest.scala diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala similarity index 100% rename from flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala rename to flink-scala/src/test/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperationTest.scala diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala index fe4b9a38570ba..91ed633dbac53 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{ConnectedStreams, DataStream} @@ -34,6 +35,7 @@ class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( * @param map2 Function called per element of the second input. * @return The resulting data stream. */ + @PublicEvolving def mapWith[R: TypeInformation](map1: IN1 => R, map2: IN2 => R): DataStream[R] = ds.map(map1, map2) @@ -48,6 +50,7 @@ class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( * @param flatMap2 Function called per element of the second input. * @return The resulting data stream. */ + @PublicEvolving def flatMapWith[R: TypeInformation]( flatMap1: IN1 => TraversableOnce[R], flatMap2: IN2 => TraversableOnce[R]): DataStream[R] = ds.flatMap(flatMap1, flatMap2) @@ -61,6 +64,7 @@ class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( * @param key2 The second stream's key function * @return The key-grouped connected streams */ + @PublicEvolving def keyingBy[K1: TypeInformation, K2: TypeInformation](key1: IN1 => K1, key2: IN2 => K2): ConnectedStreams[IN1, IN2] = ds.keyBy(key1, key2) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala index c64e7b2b2999f..124e0bc76b8ac 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} @@ -29,6 +30,7 @@ class OnDataStream[T: TypeInformation](stream: DataStream[T]) { * @tparam R The type of the items in the returned stream * @return A dataset of R */ + @PublicEvolving def mapWith[R: TypeInformation](fun: T => R): DataStream[R] = stream.map(fun) @@ -40,6 +42,7 @@ class OnDataStream[T: TypeInformation](stream: DataStream[T]) { * @tparam R The type of the items in the returned stream * @return A dataset of R */ + @PublicEvolving def flatMapWith[R: TypeInformation](fun: T => TraversableOnce[R]): DataStream[R] = stream.flatMap(fun) @@ -50,6 +53,7 @@ class OnDataStream[T: TypeInformation](stream: DataStream[T]) { * @param fun The predicate to be tested on each item * @return A dataset of R */ + @PublicEvolving def filterWith(fun: T => Boolean): DataStream[T] = stream.filter(fun) @@ -60,6 +64,7 @@ class OnDataStream[T: TypeInformation](stream: DataStream[T]) { * @tparam K The type of the key, for which type information must be known * @return A stream of Ts keyed by Ks */ + @PublicEvolving def keyingBy[K: TypeInformation](fun: T => K): KeyedStream[T, K] = stream.keyBy(fun) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index 07f4c62554714..643e127119f83 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} import org.apache.flink.streaming.api.windowing.windows.Window @@ -32,6 +33,7 @@ class OnJoinedStream[L: TypeInformation, R: TypeInformation, K, W <: Window]( * @tparam O The return type of the projection, for which type information must be known * @return A fully joined data set of Os */ + @PublicEvolving def projecting[O: TypeInformation](fun: (L, R) => O): DataStream[O] = stream.apply(fun) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala index 0c7663d29b5da..6802d91a3f5fd 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} @@ -28,6 +29,7 @@ class OnKeyedStream[T: TypeInformation, K](stream: KeyedStream[T, K]) { * @param fun The reducing function to be applied on the keyed stream * @return A data set of Ts */ + @PublicEvolving def reduceWith(fun: (T, T) => T): DataStream[T] = stream.reduce(fun) @@ -38,6 +40,7 @@ class OnKeyedStream[T: TypeInformation, K](stream: KeyedStream[T, K]) { * @param fun The reducing function to be applied on the keyed stream * @return A data set of Rs */ + @PublicEvolving def foldWith[R: TypeInformation](initialValue: R)(fun: (R, T) => R): DataStream[R] = stream.fold(initialValue)(fun) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala index 2411dfeee60e1..82021be1fe130 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions +import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, WindowedStream} import org.apache.flink.streaming.api.windowing.windows.Window @@ -38,6 +39,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { * @param function The reduce function. * @return The data stream that is the result of applying the reduce function to the window. */ + @PublicEvolving def reduceWith(function: (T, T) => T) = ds.reduce(function) @@ -49,6 +51,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { * @param function The fold function. * @return The data stream that is the result of applying the fold function to the window. */ + @PublicEvolving def foldWith[R: TypeInformation](initialValue: R)(function: (R, T) => R) = ds.fold(initialValue)(function) @@ -64,6 +67,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { * @param windowFunction The window function. * @return The data stream that is the result of applying the window function to the window. */ + @PublicEvolving def applyWith[R: TypeInformation]( initialValue: R)( foldFunction: (R, T) => R, diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index d18cd9ac7c3a6..7e10463fc4821 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -17,6 +17,7 @@ */ 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.scala.extensions.impl.acceptPartialFunctions._ import org.apache.flink.streaming.api.windowing.windows.Window @@ -63,22 +64,27 @@ import org.apache.flink.streaming.api.windowing.windows.Window */ package object extensions { + @PublicEvolving implicit def acceptPartialFunctions[T: TypeInformation](ds: DataStream[T]): OnDataStream[T] = new OnDataStream[T](ds) + @PublicEvolving implicit def acceptPartialFunctions[T: TypeInformation, K](ds: KeyedStream[T, K]): OnKeyedStream[T, K] = new OnKeyedStream[T, K](ds) + @PublicEvolving implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, K, W <: Window]( ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]): OnJoinedStream[L, R, K, W] = new OnJoinedStream[L, R, K, W](ds) + @PublicEvolving implicit def acceptPartialFunctions[IN1: TypeInformation, IN2: TypeInformation]( ds: ConnectedStreams[IN1, IN2]): OnConnectedStream[IN1, IN2] = new OnConnectedStream[IN1, IN2](ds) + @PublicEvolving implicit def acceptPartialFunctions[T, K, W <: Window]( ds: WindowedStream[T, K, W]): OnWindowedStream[T, K, W] = new OnWindowedStream[T, K, W](ds) diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedDataStreamTest.scala similarity index 100% rename from flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnConnectedDataStreamTest.scala rename to flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedDataStreamTest.scala diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStreamTest.scala similarity index 96% rename from flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala rename to flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStreamTest.scala index a415fdb790619..fb6d865932bec 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStreamTest.scala @@ -18,8 +18,7 @@ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator -import org.apache.flink.streaming.api.datastream.KeyedStream +import org.apache.flink.streaming.api.datastream.{KeyedStream, SingleOutputStreamOperator} import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedDataStreamTest.scala similarity index 100% rename from flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala rename to flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedDataStreamTest.scala index 823f61ebbfb50..34c55d7a67258 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnJoinedDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedDataStreamTest.scala @@ -19,9 +19,9 @@ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFuncti import java.util.concurrent.TimeUnit +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator import org.apache.flink.streaming.api.scala._ import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedDataStreamTest.scala similarity index 96% rename from flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala rename to flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedDataStreamTest.scala index 876a5f3f138ba..f6f153a9544b6 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnKeyedDataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedDataStreamTest.scala @@ -18,7 +18,7 @@ package org.apache.flink.streaming.api.scala.extensions.impl.acceptPartialFunctions import org.apache.flink.api.scala._ -import org.apache.flink.streaming.api.datastream.{KeyedStream, SingleOutputStreamOperator} +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions import org.apache.flink.streaming.api.scala.extensions.base.AcceptPFTestBase import org.apache.flink.streaming.api.scala.extensions.data.KeyValuePair diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedDataStreamTest.scala similarity index 100% rename from flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/acceptPartialFunctions/OnWindowedDataStreamTest.scala rename to flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedDataStreamTest.scala From 8d224534decc28dab9a37fd0c536f850adedc5bf Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Mon, 4 Apr 2016 11:49:21 +0200 Subject: [PATCH 37/39] [FLINK-1159] DataSet context bounds cleanup --- .../OnCoGroupDataSet.scala | 6 +++--- .../OnCrossDataSet.scala | 6 +++--- .../acceptPartialFunctions/OnDataSet.scala | 7 +++---- .../OnGroupedDataSet.scala | 4 ++-- .../OnHalfUnfinishedKeyPairOperation.scala | 9 ++++---- .../OnJoinFunctionAssigner.scala | 7 +++---- .../OnUnfinishedKeyPairOperation.scala | 9 ++++---- .../flink/api/scala/extensions/package.scala | 21 +++++++------------ 8 files changed, 29 insertions(+), 40 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala index 485e3525b1d07..0337d448815b6 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCoGroupDataSet.scala @@ -28,10 +28,10 @@ import scala.reflect.ClassTag * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped co-group data set - * @tparam L The type of the left data set items, for which the type information must be known - * @tparam R The type of the right data set items, for which the type information must be known + * @tparam L The type of the left data set items + * @tparam R The type of the right data set items */ -class OnCoGroupDataSet[L: TypeInformation, R: TypeInformation](ds: CoGroupDataSet[L, R]) { +class OnCoGroupDataSet[L, R](ds: CoGroupDataSet[L, R]) { /** * Co-groups the data sets using the function `fun` to project elements from both in diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala index 6a6ad76deb825..a0d4ea1f45652 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnCrossDataSet.scala @@ -28,10 +28,10 @@ import scala.reflect.ClassTag * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped cross data set - * @tparam L The type of the left data set items, for which the type information must be known - * @tparam R The type of the right data set items, for which the type information must be known + * @tparam L The type of the left data set items + * @tparam R The type of the right data set items */ -class OnCrossDataSet[L: TypeInformation, R: TypeInformation](ds: CrossDataSet[L, R]) { +class OnCrossDataSet[L, R](ds: CrossDataSet[L, R]) { /** * Starting from a cross data set, uses the function `fun` to project elements from diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala index 96c8b6ba85fbf..b2521b07a7005 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnDataSet.scala @@ -28,9 +28,9 @@ import scala.reflect.ClassTag * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped data set - * @tparam T The type of the data set items, for which the type information must be known + * @tparam T The type of the data set items */ -class OnDataSet[T: TypeInformation](ds: DataSet[T]) { +class OnDataSet[T](ds: DataSet[T]) { /** * Applies a function `fun` to each item of the data set @@ -84,11 +84,10 @@ class OnDataSet[T: TypeInformation](ds: DataSet[T]) { * Applies a reducer `fun` to the data set * * @param fun The reducing function to be applied on the whole data set - * @tparam R The type of the items in the returned collection * @return A data set of Rs */ @PublicEvolving - def reduceWith[R: TypeInformation](fun: (T, T) => T): DataSet[T] = + def reduceWith(fun: (T, T) => T): DataSet[T] = ds.reduce(fun) /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala index 71a7647fc0f88..07abccb423846 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnGroupedDataSet.scala @@ -29,9 +29,9 @@ import scala.reflect.ClassTag * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped grouped data set - * @tparam T The type of the grouped data set items, for which the type information must be known + * @tparam T The type of the grouped data set items */ -class OnGroupedDataSet[T: ClassTag](ds: GroupedDataSet[T]) { +class OnGroupedDataSet[T](ds: GroupedDataSet[T]) { /** * Sorts a group using a sorting function `fun` and an `Order` diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala index 86eb2b4eaca0a..a77c4058b7782 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnHalfUnfinishedKeyPairOperation.scala @@ -25,12 +25,11 @@ import org.apache.flink.api.scala.HalfUnfinishedKeyPairOperation * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped half unfinished key pair operation function assigner data set - * @tparam L The type of the left data set items, for which the type information must be known - * @tparam R The type of the right data set items, for which the type information must be known - * @tparam O The type of the output data set items, for which the type information must be known + * @tparam L The type of the left data set items + * @tparam R The type of the right data set items + * @tparam O The type of the output data set items */ -class OnHalfUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( - ds: HalfUnfinishedKeyPairOperation[L, R, O]) { +class OnHalfUnfinishedKeyPairOperation[L, R, O](ds: HalfUnfinishedKeyPairOperation[L, R, O]) { /** * Initiates a join or co-group operation, defining the second half of diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala index ce24e92b7aa79..4ab41e5d56742 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnJoinFunctionAssigner.scala @@ -28,11 +28,10 @@ import scala.reflect.ClassTag * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped join function assigner data set - * @tparam L The type of the left data set items, for which the type information must be known - * @tparam R The type of the right data set items, for which the type information must be known + * @tparam L The type of the left data set items + * @tparam R The type of the right data set items */ -class OnJoinFunctionAssigner[L: TypeInformation, R: TypeInformation]( - ds: JoinFunctionAssigner[L, R]) { +class OnJoinFunctionAssigner[L, R](ds: JoinFunctionAssigner[L, R]) { /** * Joins the data sets using the function `fun` to project elements from both in the diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala index 69bad019f446d..4fa6fcc238dff 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/impl/acceptPartialFunctions/OnUnfinishedKeyPairOperation.scala @@ -26,12 +26,11 @@ import org.apache.flink.api.scala.{HalfUnfinishedKeyPairOperation, UnfinishedKey * perform extraction of items in a tuple, case class instance or collection * * @param ds The wrapped unfinished key pair operation data set - * @tparam L The type of the left data set items, for which the type information must be known - * @tparam R The type of the right data set items, for which the type information must be known - * @tparam O The type of the output data set items, for which the type information must be known + * @tparam L The type of the left data set items + * @tparam R The type of the right data set items + * @tparam O The type of the output data set items */ -class OnUnfinishedKeyPairOperation[L: TypeInformation, R: TypeInformation, O: TypeInformation]( - ds: UnfinishedKeyPairOperation[L, R, O]) { +class OnUnfinishedKeyPairOperation[L, R, O](ds: UnfinishedKeyPairOperation[L, R, O]) { /** * Initiates a join or co-group operation, defining the first half of diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala index 56f16f22e0bee..7e5ab8ac5eb5b 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala @@ -18,11 +18,8 @@ package org.apache.flink.api.scala import org.apache.flink.annotation.PublicEvolving -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.extensions.impl.acceptPartialFunctions._ -import scala.reflect.ClassTag - /** * acceptPartialFunctions extends the original DataSet with methods with unique names * that delegate to core higher-order functions (e.g. `map`) so that we can work around @@ -65,37 +62,33 @@ import scala.reflect.ClassTag package object extensions { @PublicEvolving - implicit def acceptPartialFunctions[T: TypeInformation](ds: DataSet[T]): OnDataSet[T] = + implicit def acceptPartialFunctions[T](ds: DataSet[T]): OnDataSet[T] = new OnDataSet[T](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( + implicit def acceptPartialFunctions[L, R]( ds: JoinFunctionAssigner[L, R]): OnJoinFunctionAssigner[L, R] = new OnJoinFunctionAssigner[L, R](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( - ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = + implicit def acceptPartialFunctions[L, R](ds: CrossDataSet[L, R]): OnCrossDataSet[L, R] = new OnCrossDataSet[L, R](ds) @PublicEvolving - implicit def acceptPartialFunctions[T: TypeInformation: ClassTag]( - ds: GroupedDataSet[T]): - OnGroupedDataSet[T] = + implicit def acceptPartialFunctions[T](ds: GroupedDataSet[T]): OnGroupedDataSet[T] = new OnGroupedDataSet[T](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation]( - ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = + implicit def acceptPartialFunctions[L, R](ds: CoGroupDataSet[L, R]): OnCoGroupDataSet[L, R] = new OnCoGroupDataSet[L, R](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + implicit def acceptPartialFunctions[L, R, O]( ds: HalfUnfinishedKeyPairOperation[L, R, O]): OnHalfUnfinishedKeyPairOperation[L, R, O] = new OnHalfUnfinishedKeyPairOperation[L, R, O](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, O: TypeInformation]( + implicit def acceptPartialFunctions[L, R, O]( ds: UnfinishedKeyPairOperation[L, R, O]): OnUnfinishedKeyPairOperation[L, R, O] = new OnUnfinishedKeyPairOperation[L, R, O](ds) From 4a721828894342b5c91f56b5965ab978977f8e0e Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Mon, 4 Apr 2016 12:20:20 +0200 Subject: [PATCH 38/39] [FLINK-1159] DataStream context bounds cleanup --- .../acceptPartialFunctions/OnConnectedStream.scala | 3 +-- .../impl/acceptPartialFunctions/OnDataStream.scala | 2 +- .../impl/acceptPartialFunctions/OnJoinedStream.scala | 2 +- .../impl/acceptPartialFunctions/OnKeyedStream.scala | 2 +- .../flink/streaming/api/scala/extensions/package.scala | 10 ++++------ 5 files changed, 8 insertions(+), 11 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala index 91ed633dbac53..fea369d8d78bf 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala @@ -21,8 +21,7 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{ConnectedStreams, DataStream} -class OnConnectedStream[IN1: TypeInformation, IN2: TypeInformation]( - ds: ConnectedStreams[IN1, IN2]) { +class OnConnectedStream[IN1, IN2](ds: ConnectedStreams[IN1, IN2]) { /** * Applies a CoMap transformation on the connected streams. diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala index 124e0bc76b8ac..ffad84ceb3334 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala @@ -21,7 +21,7 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} -class OnDataStream[T: TypeInformation](stream: DataStream[T]) { +class OnDataStream[T](stream: DataStream[T]) { /** * Applies a function `fun` to each item of the stream diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index 643e127119f83..bc26cc8d759ef 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} import org.apache.flink.streaming.api.windowing.windows.Window -class OnJoinedStream[L: TypeInformation, R: TypeInformation, K, W <: Window]( +class OnJoinedStream[L, R, K, W <: Window]( stream: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) { /** diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala index 6802d91a3f5fd..68b94d7fff1e2 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala @@ -21,7 +21,7 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} -class OnKeyedStream[T: TypeInformation, K](stream: KeyedStream[T, K]) { +class OnKeyedStream[T, K](stream: KeyedStream[T, K]) { /** * Applies a reducer `fun` to the stream diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index 7e10463fc4821..69e8e4ed06c08 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -65,22 +65,20 @@ import org.apache.flink.streaming.api.windowing.windows.Window package object extensions { @PublicEvolving - implicit def acceptPartialFunctions[T: TypeInformation](ds: DataStream[T]): - OnDataStream[T] = + implicit def acceptPartialFunctions[T](ds: DataStream[T]): OnDataStream[T] = new OnDataStream[T](ds) @PublicEvolving - implicit def acceptPartialFunctions[T: TypeInformation, K](ds: KeyedStream[T, K]): - OnKeyedStream[T, K] = + implicit def acceptPartialFunctions[T, K](ds: KeyedStream[T, K]): OnKeyedStream[T, K] = new OnKeyedStream[T, K](ds) @PublicEvolving - implicit def acceptPartialFunctions[L: TypeInformation, R: TypeInformation, K, W <: Window]( + implicit def acceptPartialFunctions[L, R, K, W <: Window]( ds: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]): OnJoinedStream[L, R, K, W] = new OnJoinedStream[L, R, K, W](ds) @PublicEvolving - implicit def acceptPartialFunctions[IN1: TypeInformation, IN2: TypeInformation]( + implicit def acceptPartialFunctions[IN1, IN2]( ds: ConnectedStreams[IN1, IN2]): OnConnectedStream[IN1, IN2] = new OnConnectedStream[IN1, IN2](ds) From 900058f527cf755ea2ba441d6a316e6c50158825 Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Mon, 4 Apr 2016 12:31:26 +0200 Subject: [PATCH 39/39] [FLINK-1159] Add missing Scaladoc to DataStream extension --- .../OnConnectedStream.scala | 16 ++++++++++++---- .../acceptPartialFunctions/OnDataStream.scala | 7 +++++++ .../acceptPartialFunctions/OnJoinedStream.scala | 10 ++++++++++ .../acceptPartialFunctions/OnKeyedStream.scala | 8 ++++++++ .../OnWindowedStream.scala | 17 +++++++++++++---- .../api/scala/extensions/package.scala | 1 - 6 files changed, 50 insertions(+), 9 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala index fea369d8d78bf..deb03a3beb222 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnConnectedStream.scala @@ -21,7 +21,15 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{ConnectedStreams, DataStream} -class OnConnectedStream[IN1, IN2](ds: ConnectedStreams[IN1, IN2]) { +/** + * Wraps a connected data stream, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param stream The wrapped data stream + * @tparam IN1 The type of the data stream items coming from the first connection + * @tparam IN2 The type of the data stream items coming from the second connection + */ +class OnConnectedStream[IN1, IN2](stream: ConnectedStreams[IN1, IN2]) { /** * Applies a CoMap transformation on the connected streams. @@ -36,7 +44,7 @@ class OnConnectedStream[IN1, IN2](ds: ConnectedStreams[IN1, IN2]) { */ @PublicEvolving def mapWith[R: TypeInformation](map1: IN1 => R, map2: IN2 => R): DataStream[R] = - ds.map(map1, map2) + stream.map(map1, map2) /** * Applies a CoFlatMap transformation on the connected streams. @@ -52,7 +60,7 @@ class OnConnectedStream[IN1, IN2](ds: ConnectedStreams[IN1, IN2]) { @PublicEvolving def flatMapWith[R: TypeInformation]( flatMap1: IN1 => TraversableOnce[R], flatMap2: IN2 => TraversableOnce[R]): DataStream[R] = - ds.flatMap(flatMap1, flatMap2) + stream.flatMap(flatMap1, flatMap2) /** * Keys the two connected streams together. After this operation, all @@ -66,6 +74,6 @@ class OnConnectedStream[IN1, IN2](ds: ConnectedStreams[IN1, IN2]) { @PublicEvolving def keyingBy[K1: TypeInformation, K2: TypeInformation](key1: IN1 => K1, key2: IN2 => K2): ConnectedStreams[IN1, IN2] = - ds.keyBy(key1, key2) + stream.keyBy(key1, key2) } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala index ffad84ceb3334..8d98c46630045 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnDataStream.scala @@ -21,6 +21,13 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} +/** + * Wraps a data stream, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param stream The wrapped data stream + * @tparam T The type of the data stream items + */ class OnDataStream[T](stream: DataStream[T]) { /** diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala index bc26cc8d759ef..226eb8b029ac4 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnJoinedStream.scala @@ -22,6 +22,16 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, JoinedStreams} import org.apache.flink.streaming.api.windowing.windows.Window +/** + * Wraps a joined data stream, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param stream The wrapped data stream + * @tparam L The type of the data stream items from the left side of the join + * @tparam R The type of the data stream items from the right input of the join + * @tparam K The type of key + * @tparam W The type of the window + */ class OnJoinedStream[L, R, K, W <: Window]( stream: JoinedStreams[L, R]#Where[K]#EqualTo#WithWindow[W]) { diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala index 68b94d7fff1e2..218bcbfd6553c 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnKeyedStream.scala @@ -21,6 +21,14 @@ import org.apache.flink.annotation.PublicEvolving import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, KeyedStream} +/** + * Wraps a keyed data stream, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param stream The wrapped data stream + * @tparam T The type of the data stream items + * @tparam K The type of key + */ class OnKeyedStream[T, K](stream: KeyedStream[T, K]) { /** diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala index 82021be1fe130..f7a5923c2894c 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/impl/acceptPartialFunctions/OnWindowedStream.scala @@ -22,7 +22,16 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.{DataStream, WindowedStream} import org.apache.flink.streaming.api.windowing.windows.Window -class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { +/** + * Wraps a joined data stream, allowing to use anonymous partial functions to + * perform extraction of items in a tuple, case class instance or collection + * + * @param stream The wrapped data stream + * @tparam T The type of the data stream items from the right input of the join + * @tparam K The type of key + * @tparam W The type of the window + */ +class OnWindowedStream[T, K, W <: Window](stream: WindowedStream[T, K, W]) { /** * Applies a reduce function to the window. The window function is called for each evaluation @@ -41,7 +50,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { */ @PublicEvolving def reduceWith(function: (T, T) => T) = - ds.reduce(function) + stream.reduce(function) /** * Applies the given fold function to each window. The window function is called for each @@ -53,7 +62,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { */ @PublicEvolving def foldWith[R: TypeInformation](initialValue: R)(function: (R, T) => R) = - ds.fold(initialValue)(function) + stream.fold(initialValue)(function) /** * Applies the given window function to each window. The window function is called for each @@ -73,7 +82,7 @@ class OnWindowedStream[T, K, W <: Window](ds: WindowedStream[T, K, W]) { foldFunction: (R, T) => R, windowFunction: (K, W, Stream[R]) => TraversableOnce[R]) : DataStream[R] = - ds.apply(initialValue, foldFunction, { + stream.apply(initialValue, foldFunction, { (key, window, items, out) => windowFunction(key, window, items.toStream).foreach(out.collect) }) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala index 69e8e4ed06c08..f82bb7407285e 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/extensions/package.scala @@ -18,7 +18,6 @@ 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.scala.extensions.impl.acceptPartialFunctions._ import org.apache.flink.streaming.api.windowing.windows.Window