From 9bdc93166a9dfc25b0e83cb86769151281663563 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 18 Jul 2014 15:37:19 +0200 Subject: [PATCH 01/14] Renamed of Function to RichFunction, created empty Function interface --- ...unction.java => AbstractRichFunction.java} | 2 +- ...nericFilter.java => FilterFunctional.java} | 2 +- .../flink/api/common/functions/Function.java | 92 +----------------- .../api/common/functions/RichFunction.java | 97 +++++++++++++++++++ 4 files changed, 100 insertions(+), 93 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{AbstractFunction.java => AbstractRichFunction.java} (97%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericFilter.java => FilterFunctional.java} (95%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java similarity index 97% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractFunction.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java index f4b276341354e..07b957d3ef99e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java @@ -28,7 +28,7 @@ * for {@link #open(Configuration)} and {@link #close()}. It also offers access to the * {@link RuntimeContext} and {@link IterationRuntimeContext}. */ -public abstract class AbstractFunction implements Function, Serializable { +public abstract class AbstractRichFunction implements RichFunction, Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFilter.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFilter.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java index f34b0385546de..c5ba78f0cf582 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFilter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java @@ -20,7 +20,7 @@ package org.apache.flink.api.common.functions; -public interface GenericFilter extends Function { +public interface FilterFunctional extends Function { /** * User defined function for a filter. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java index d3b7db48906be..82e964bb95a4f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java @@ -1,97 +1,7 @@ -/** - * 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.common.functions; -import org.apache.flink.configuration.Configuration; - /** - * An base interface for all user-defined functions. This class defines methods for - * the life cycle of the functions, as well as methods to access the context in which the functions - * are executed. + * Created by kostas on 18/07/14. */ public interface Function { - - /** - * Initialization method for the function. It is called before the actual working methods - * (like map or join) and thus suitable for one time setup work. For functions that - * are part of an iteration, this method will be invoked at the beginning of each iteration superstep. - *

- * The configuration object passed to the function can be used for configuration and initialization. - * The configuration contains all parameters that were configured on the function in the program - * composition. - * - *

- * public class MyMapper extends FilterFunction { - * - * private String searchString; - * - * public void open(Configuration parameters) { - * this.searchString = parameters.getString("foo"); - * } - * - * public boolean filter(String value) { - * return value.equals(searchString); - * } - * } - *
- *

- * By default, this method does nothing. - * - * @param parameters The configuration containing the parameters attached to the contract. - * - * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the - * runtime catches an exception, it aborts the task and lets the fail-over logic - * decide whether to retry the task execution. - * - * @see org.apache.flink.configuration.Configuration - */ - void open(Configuration parameters) throws Exception; - - /** - * Teardown method for the user code. It is called after the last call to the main working methods - * (e.g. map or join). For functions that are part of an iteration, this method will - * be invoked after each iteration superstep. - *

- * This method can be used for clean up work. - * - * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the - * runtime catches an exception, it aborts the task and lets the fail-over logic - * decide whether to retry the task execution. - */ - void close() throws Exception; - - - /** - * Gets the context that contains information about the UDF's runtime. - * - * Context information are for example {@link org.apache.flink.api.common.accumulators.Accumulator}s - * or the {@link org.apache.flink.api.common.cache.DistributedCache}. - * - * @return The UDF's runtime context. - */ - RuntimeContext getRuntimeContext(); - - /** - * Sets the function's runtime context. Called by the framework when creating a parallel instance of the function. - * - * @param t The runtime context. - */ - void setRuntimeContext(RuntimeContext t); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java new file mode 100644 index 0000000000000..a421f7fcf519a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java @@ -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.api.common.functions; + +import org.apache.flink.configuration.Configuration; + +/** + * An base interface for all user-defined functions. This class defines methods for + * the life cycle of the functions, as well as methods to access the context in which the functions + * are executed. + */ +public interface RichFunction { + + /** + * Initialization method for the function. It is called before the actual working methods + * (like map or join) and thus suitable for one time setup work. For functions that + * are part of an iteration, this method will be invoked at the beginning of each iteration superstep. + *

+ * The configuration object passed to the function can be used for configuration and initialization. + * The configuration contains all parameters that were configured on the function in the program + * composition. + * + *

+ * public class MyMapper extends FilterFunction { + * + * private String searchString; + * + * public void open(Configuration parameters) { + * this.searchString = parameters.getString("foo"); + * } + * + * public boolean filter(String value) { + * return value.equals(searchString); + * } + * } + *
+ *

+ * By default, this method does nothing. + * + * @param parameters The configuration containing the parameters attached to the contract. + * + * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the + * runtime catches an exception, it aborts the task and lets the fail-over logic + * decide whether to retry the task execution. + * + * @see org.apache.flink.configuration.Configuration + */ + void open(Configuration parameters) throws Exception; + + /** + * Teardown method for the user code. It is called after the last call to the main working methods + * (e.g. map or join). For functions that are part of an iteration, this method will + * be invoked after each iteration superstep. + *

+ * This method can be used for clean up work. + * + * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the + * runtime catches an exception, it aborts the task and lets the fail-over logic + * decide whether to retry the task execution. + */ + void close() throws Exception; + + + /** + * Gets the context that contains information about the UDF's runtime. + * + * Context information are for example {@link org.apache.flink.api.common.accumulators.Accumulator}s + * or the {@link org.apache.flink.api.common.cache.DistributedCache}. + * + * @return The UDF's runtime context. + */ + RuntimeContext getRuntimeContext(); + + /** + * Sets the function's runtime context. Called by the framework when creating a parallel instance of the function. + * + * @param t The runtime context. + */ + void setRuntimeContext(RuntimeContext t); +} From d8bdd90bc35446d65d8a0bbb3a75daa23f9f5b2a Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 18 Jul 2014 15:37:26 +0200 Subject: [PATCH 02/14] Renamed of Function to RichFunction, created empty Function interface --- .../org/apache/flink/compiler/util/NoOpFunction.java | 4 ++-- .../org/apache/flink/compiler/CompilerTestBase.java | 8 +++----- .../apache/flink/api/common/functions/Function.java | 5 ++++- .../flink/api/common/functions/GenericCoGrouper.java | 2 +- .../api/common/functions/GenericCollectorMap.java | 2 +- .../flink/api/common/functions/GenericCombine.java | 2 +- .../flink/api/common/functions/GenericCrosser.java | 2 +- .../flink/api/common/functions/GenericFlatMap.java | 2 +- .../api/common/functions/GenericGroupReduce.java | 2 +- .../flink/api/common/functions/GenericJoiner.java | 2 +- .../flink/api/common/functions/GenericMap.java | 2 +- .../flink/api/common/functions/GenericReduce.java | 2 +- .../flink/api/common/functions/RichFunction.java | 4 ++-- .../flink/api/common/functions/RuntimeContext.java | 2 +- .../api/common/operators/AbstractUdfOperator.java | 4 ++-- .../api/common/operators/DualInputOperator.java | 4 ++-- .../api/common/operators/SingleInputOperator.java | 4 ++-- .../org/apache/flink/api/common/operators/Union.java | 6 +++--- .../api/common/operators/base/BulkIterationBase.java | 8 ++++---- .../common/operators/base/DeltaIterationBase.java | 6 +++--- .../api/common/operators/util/OperatorUtil.java | 2 +- .../api/common/operators/util/OperatorUtilTest.java | 5 ++--- .../main/java/org/apache/flink/api/java/DataSet.java | 3 ++- .../org/apache/flink/api/java/DeltaIteration.java | 2 +- .../apache/flink/api/java/ExecutionEnvironment.java | 4 ++-- .../org/apache/flink/api/java/IterativeDataSet.java | 2 +- .../flink/api/java/functions/CoGroupFunction.java | 4 ++-- .../flink/api/java/functions/CrossFunction.java | 4 ++-- .../flink/api/java/functions/FilterFunction.java | 6 +++--- .../flink/api/java/functions/FlatMapFunction.java | 4 ++-- .../api/java/functions/GroupReduceFunction.java | 4 ++-- .../flink/api/java/functions/JoinFunction.java | 4 ++-- .../apache/flink/api/java/functions/MapFunction.java | 4 ++-- .../flink/api/java/functions/ReduceFunction.java | 4 ++-- .../flink/api/java/operators/FilterOperator.java | 5 +++-- .../apache/flink/api/java/operators/UdfOperator.java | 6 +++--- .../operators/translation/PlanFilterOperator.java | 5 +++-- .../operators/translation/PlanProjectOperator.java | 4 ++-- .../java/operators/translation/WrappingFunction.java | 4 ++-- .../api/java/record/functions/CoGroupFunction.java | 4 ++-- .../api/java/record/functions/CrossFunction.java | 4 ++-- .../api/java/record/functions/JoinFunction.java | 4 ++-- .../flink/api/java/record/functions/MapFunction.java | 4 ++-- .../api/java/record/functions/ReduceFunction.java | 4 ++-- .../iterative/task/AbstractIterativePactTask.java | 4 ++-- .../iterative/task/IterationHeadPactTask.java | 4 ++-- .../task/IterationIntermediatePactTask.java | 4 ++-- .../iterative/task/IterationTailPactTask.java | 4 ++-- .../apache/flink/runtime/operators/NoOpDriver.java | 10 +++++----- .../apache/flink/runtime/operators/PactDriver.java | 4 ++-- .../flink/runtime/operators/RegularPactTask.java | 12 ++++++------ .../runtime/operators/ResettablePactDriver.java | 4 ++-- .../chaining/ChainedCollectorMapDriver.java | 4 ++-- .../runtime/operators/chaining/ChainedDriver.java | 4 ++-- .../operators/chaining/ChainedFlatMapDriver.java | 4 ++-- .../runtime/operators/chaining/ChainedMapDriver.java | 4 ++-- .../chaining/ChainedTerminationCriterionDriver.java | 4 ++-- .../chaining/SynchronousChainedCombineDriver.java | 4 ++-- .../flink/runtime/operators/util/TaskConfig.java | 4 ++-- .../operators/hash/HashMatchIteratorITCase.java | 6 ++---- .../runtime/operators/testutils/DriverTestBase.java | 4 ++-- .../runtime/operators/testutils/TaskTestBase.java | 6 +++--- .../flink/api/scala/operators/IterateOperators.scala | 2 +- .../flink/test/compiler/util/CompilerTestBase.java | 4 ++-- .../flink/test/compiler/util/OperatorResolver.java | 4 ++-- .../CustomCompensatableDotProductCoGroup.java | 4 ++-- .../CustomCompensatableDotProductMatch.java | 4 ++-- .../CustomCompensatingMap.java | 4 ++-- .../customdanglingpagerank/CustomRankCombiner.java | 4 ++-- .../flink/test/recordJobs/relational/TPCHQuery3.java | 2 +- 70 files changed, 144 insertions(+), 143 deletions(-) diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpFunction.java b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpFunction.java index 9f10be5ff2cb9..6eccc8a130507 100644 --- a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpFunction.java +++ b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpFunction.java @@ -18,9 +18,9 @@ package org.apache.flink.compiler.util; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; -public class NoOpFunction extends AbstractFunction { +public class NoOpFunction extends AbstractRichFunction { private static final long serialVersionUID = 1L; } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java index ff4d6b04aac20..b2c163b4ec0f3 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java @@ -27,14 +27,12 @@ import java.util.Set; import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.GenericDataSourceBase; import org.apache.flink.api.java.record.operators.BulkIteration; import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.compiler.DataStatistics; -import org.apache.flink.compiler.PactCompiler; import org.apache.flink.compiler.costs.DefaultCostEstimator; import org.apache.flink.compiler.plan.OptimizedPlan; import org.apache.flink.compiler.plan.PlanNode; @@ -181,7 +179,7 @@ public T getNode(String name) { } @SuppressWarnings("unchecked") - public T getNode(String name, Class stubClass) { + public T getNode(String name, Class stubClass) { List nodes = this.map.get(name); if (nodes == null || nodes.isEmpty()) { throw new RuntimeException("No node found with the given name and stub class."); @@ -243,7 +241,7 @@ public > T getNode(String name) { } @SuppressWarnings("unchecked") - public > T getNode(String name, Class stubClass) { + public > T getNode(String name, Class stubClass) { List> nodes = this.map.get(name); if (nodes == null || nodes.isEmpty()) { throw new RuntimeException("No node found with the given name and stub class."); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java index 82e964bb95a4f..00ea0924ba7e5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java @@ -1,7 +1,10 @@ package org.apache.flink.api.common.functions; /** - * Created by kostas on 18/07/14. + * An base interface for all user-defined functions. This interface is empty in order + * to enable functions that are SAM (single abstract method) interfaces, so that they + * can be called as Java 8 lambdas */ public interface Function { + } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java index 59669a242a509..da4ccb5e0493e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java @@ -24,7 +24,7 @@ import org.apache.flink.util.Collector; -public interface GenericCoGrouper extends Function { +public interface GenericCoGrouper extends RichFunction { /** * This method must be implemented to provide a user implementation of a diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCollectorMap.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCollectorMap.java index ada4eebf6cfdb..41cfa1d894d51 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCollectorMap.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCollectorMap.java @@ -23,7 +23,7 @@ -public interface GenericCollectorMap extends Function { +public interface GenericCollectorMap extends RichFunction { void map(T record, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java index 8dfe758b68f28..03d028b5bea4d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java @@ -26,7 +26,7 @@ /** * Generic interface used for combiners. */ -public interface GenericCombine extends Function { +public interface GenericCombine extends RichFunction { void combine(Iterator records, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java index 3de9b1d598f3d..e4663479612e1 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java @@ -27,7 +27,7 @@ * @param Second input type * @param Output type */ -public interface GenericCrosser extends Function { +public interface GenericCrosser extends RichFunction { /** * User defined function for the cross operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java index efb1d49d48636..603da1cdb1b07 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java @@ -27,7 +27,7 @@ * @param * @param */ -public interface GenericFlatMap extends Function { +public interface GenericFlatMap extends RichFunction { /** * User defined function to perform transformations on records. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java index e8d9910b433ce..fe616a4605ad3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java @@ -29,7 +29,7 @@ * @param Incoming types * @param Outgoing types */ -public interface GenericGroupReduce extends Function { +public interface GenericGroupReduce extends RichFunction { /** * * The central function to be implemented for a reducer. The function receives per call one diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java index 77c2ac9081db0..bfa6a278bf04e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java @@ -22,7 +22,7 @@ import org.apache.flink.util.Collector; -public interface GenericJoiner extends Function { +public interface GenericJoiner extends RichFunction { void join(V1 value1, V2 value2, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java index 316bf5dbf4aa5..d022e95fe1b21 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java @@ -20,7 +20,7 @@ package org.apache.flink.api.common.functions; -public interface GenericMap extends Function { +public interface GenericMap extends RichFunction { /** * A user-implemented function that modifies or transforms an incoming object and diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java index 9e75f2e47a3ac..db01d949609f0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java @@ -20,7 +20,7 @@ package org.apache.flink.api.common.functions; -public interface GenericReduce extends Function { +public interface GenericReduce extends RichFunction { T reduce(T value1, T value2) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java index a421f7fcf519a..ffc3ac2175cb1 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java @@ -22,11 +22,11 @@ import org.apache.flink.configuration.Configuration; /** - * An base interface for all user-defined functions. This class defines methods for + * An base interface for all rich user-defined functions. This class defines methods for * the life cycle of the functions, as well as methods to access the context in which the functions * are executed. */ -public interface RichFunction { +public interface RichFunction extends Function { /** * Initialization method for the function. It is called before the actual working methods diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index a5f0b19e199ed..3cf30ffc480f2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -34,7 +34,7 @@ * the current degree of parallelism) and other constructs like accumulators and broadcast variables. *

* A function can, during runtime, obtain the RuntimeContext via a call to - * {@link org.apache.flink.api.common.functions.AbstractFunction#getRuntimeContext()}. + * {@link AbstractRichFunction#getRuntimeContext()}. */ public interface RuntimeContext { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java index 055765aa87cf7..15243044fc6ee 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java @@ -22,7 +22,7 @@ import java.util.HashMap; import java.util.Map; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** @@ -30,7 +30,7 @@ * * @param Type of the user function */ -public abstract class AbstractUdfOperator extends Operator { +public abstract class AbstractUdfOperator extends Operator { /** * The object or class containing the user function. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java index e57c90d230271..82b6b3064f395 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.util.Visitor; @@ -33,7 +33,7 @@ * @param Output type of the user function * @param Type of the user function */ -public abstract class DualInputOperator extends AbstractUdfOperator { +public abstract class DualInputOperator extends AbstractUdfOperator { /** * The operator producing the first input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java index 38c68f5971356..9950cf67818a7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.util.Visitor; @@ -32,7 +32,7 @@ * @param Output type of the user function * @param Type of the user function */ -public abstract class SingleInputOperator extends AbstractUdfOperator { +public abstract class SingleInputOperator extends AbstractUdfOperator { /** * The input which produces the data consumed by this operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java index b140ddab9f4d7..c4167657ebc1f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java @@ -19,13 +19,13 @@ package org.apache.flink.api.common.operators; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; /** * This operator represents a Union between two inputs. */ -public class Union extends DualInputOperator { +public class Union extends DualInputOperator { private final static String NAME = "Union"; @@ -34,7 +34,7 @@ public class Union extends DualInputOperator { */ public Union(BinaryOperatorInformation operatorInfo) { // we pass it an AbstractFunction, because currently all operators expect some form of UDF - super(new UserCodeClassWrapper(AbstractFunction.class), operatorInfo, NAME); + super(new UserCodeClassWrapper(AbstractRichFunction.class), operatorInfo, NAME); } public Union(Operator input1, Operator input2) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java index ffcab50c2066e..66bea7f08e3bb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java @@ -29,7 +29,7 @@ import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.aggregators.AggregatorRegistry; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.operators.IterationOperator; import org.apache.flink.api.common.operators.Operator; @@ -48,7 +48,7 @@ /** * */ -public class BulkIterationBase extends SingleInputOperator implements IterationOperator { +public class BulkIterationBase extends SingleInputOperator implements IterationOperator { private static String DEFAULT_NAME = ""; @@ -78,7 +78,7 @@ public BulkIterationBase(UnaryOperatorInformation operatorInfo) { * @param name */ public BulkIterationBase(UnaryOperatorInformation operatorInfo, String name) { - super(new UserCodeClassWrapper(AbstractFunction.class), operatorInfo, name); + super(new UserCodeClassWrapper(AbstractRichFunction.class), operatorInfo, name); inputPlaceHolder = new PartialSolutionPlaceHolder(this, this.getOperatorInfo()); } @@ -230,7 +230,7 @@ public UserCodeWrapper getUserCodeWrapper() { /** * Special Mapper that is added before a termination criterion and is only a container for an special aggregator */ - public static class TerminationCriterionMapper extends AbstractFunction implements Serializable, GenericCollectorMap { + public static class TerminationCriterionMapper extends AbstractRichFunction implements Serializable, GenericCollectorMap { private static final long serialVersionUID = 1L; private TerminationCriterionAggregator aggregator; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java index 89e5008e74ac4..8e955b1341191 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java @@ -23,7 +23,7 @@ import java.util.Map; import org.apache.flink.api.common.aggregators.AggregatorRegistry; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.IterationOperator; @@ -51,7 +51,7 @@ * This class is a subclass of {@code DualInputOperator}. The solution set is considered the first input, the * workset is considered the second input. */ -public class DeltaIterationBase extends DualInputOperator implements IterationOperator { +public class DeltaIterationBase extends DualInputOperator implements IterationOperator { private final Operator solutionSetPlaceholder; @@ -88,7 +88,7 @@ public DeltaIterationBase(BinaryOperatorInformation operatorInfo, in } public DeltaIterationBase(BinaryOperatorInformation operatorInfo, int[] keyPositions, String name) { - super(new UserCodeClassWrapper(AbstractFunction.class), operatorInfo, name); + super(new UserCodeClassWrapper(AbstractRichFunction.class), operatorInfo, name); this.solutionSetKeyFields = keyPositions; solutionSetPlaceholder = new SolutionSetPlaceHolder(this, new OperatorInformation(operatorInfo.getFirstInputType())); worksetPlaceholder = new WorksetPlaceHolder(this, new OperatorInformation(operatorInfo.getSecondInputType())); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java index 2ebae24f4b630..18097c9afba84 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java @@ -67,7 +67,7 @@ public class OperatorUtil { } /** - * Returns the associated {@link Operator} type for the given {@link org.apache.flink.api.common.functions.Function} class. + * Returns the associated {@link Operator} type for the given {@link org.apache.flink.api.common.functions.RichFunction} class. * * @param stubClass * the stub class diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java index 647ceab23470c..446f4e3d19fc3 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.functions.GenericCrosser; @@ -36,7 +36,6 @@ import org.apache.flink.api.common.operators.base.GenericDataSourceBase; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.operators.base.JoinOperatorBase; -import org.apache.flink.api.common.operators.util.OperatorUtil; import org.apache.flink.types.IntValue; import org.junit.Test; @@ -85,7 +84,7 @@ public void getContractClassShouldReturnMatchForMatchStub() { */ @Test public void getContractClassShouldReturnNullForStub() { - final Class result = OperatorUtil.getContractClass(Function.class); + final Class result = OperatorUtil.getContractClass(RichFunction.class); assertEquals(null, result); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 894880ee42879..44ed8eb7a9e3d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.Validate; +import org.apache.flink.api.common.functions.FilterFunctional; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.java.aggregation.Aggregations; @@ -173,7 +174,7 @@ public FlatMapOperator flatMap(FlatMapFunction flatMapper) { * @see FilterOperator * @see DataSet */ - public FilterOperator filter(FilterFunction filter) { + public FilterOperator filter(FilterFunctional filter) { if (filter == null) { throw new NullPointerException("Filter function must not be null."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java index bb53a895195d0..2f2eae0a4a1a6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java @@ -189,7 +189,7 @@ public int getParallelism() { * The value of an aggregator can be accessed in the next iteration. *

* Aggregators can be accessed inside a function via the - * {@link org.apache.flink.api.common.functions.AbstractFunction#getIterationRuntimeContext()} method. + * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method. * * @param name The name under which the aggregator is registered. * @param aggregator The aggregator class. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index d00fb4743323a..3c3b9a0aa6843 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -550,7 +550,7 @@ public JobExecutionResult execute() throws Exception { * The runtime will copy the files temporarily to a local cache, if needed. *

* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.Function#getRuntimeContext()} and provides access + * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access * {@link org.apache.flink.api.common.cache.DistributedCache} via * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. * @@ -568,7 +568,7 @@ public void registerCachedFile(String filePath, String name){ * The runtime will copy the files temporarily to a local cache, if needed. *

* The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.Function#getRuntimeContext()} and provides access + * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access * {@link org.apache.flink.api.common.cache.DistributedCache} via * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. * diff --git a/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java index 6443dbb87e7a3..5dc0d2e9ce77e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java @@ -93,7 +93,7 @@ public int getMaxIterations() { * The value of an aggregator can be accessed in the next iteration. *

* Aggregators can be accessed inside a function via the - * {@link org.apache.flink.api.common.functions.AbstractFunction#getIterationRuntimeContext()} method. + * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method. * * @param name The name under which the aggregator is registered. * @param aggregator The aggregator class. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java index 201794abce4a8..87cf308e10d39 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java @@ -20,7 +20,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.util.Collector; @@ -50,7 +50,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CoGroupFunction extends AbstractFunction implements GenericCoGrouper { +public abstract class CoGroupFunction extends AbstractRichFunction implements GenericCoGrouper { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java index 27907ec0c9e0c..50096a70f3aec 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCrosser; import org.apache.flink.util.Collector; @@ -46,7 +46,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CrossFunction extends AbstractFunction implements GenericCrosser{ +public abstract class CrossFunction extends AbstractRichFunction implements GenericCrosser{ private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java index aac20865e3578..ff9ae452d8836 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; -import org.apache.flink.api.common.functions.GenericFilter; +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.FilterFunctional; /** * The abstract base class for Filter functions. A filter function take elements and evaluates a @@ -36,7 +36,7 @@ * * @param The type of the filtered elements. */ -public abstract class FilterFunction extends AbstractFunction implements GenericFilter { +public abstract class FilterFunction extends AbstractRichFunction implements FilterFunctional { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java index f9c22ccfd9fa7..321c6bf18991e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.util.Collector; @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class FlatMapFunction extends AbstractFunction implements GenericFlatMap { +public abstract class FlatMapFunction extends AbstractRichFunction implements GenericFlatMap { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java index 01ae9c1addbef..562e8bb88f35c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java @@ -24,7 +24,7 @@ import java.lang.annotation.Target; import java.util.Iterator; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.util.Collector; @@ -51,7 +51,7 @@ * @param Type of the elements that this function processes. * @param The type of the elements returned by the user-defined function. */ -public abstract class GroupReduceFunction extends AbstractFunction implements GenericGroupReduce, GenericCombine { +public abstract class GroupReduceFunction extends AbstractRichFunction implements GenericGroupReduce, GenericCombine { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java index c78e6f3e95327..0624efd73f72e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.util.Collector; @@ -55,7 +55,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class JoinFunction extends AbstractFunction implements GenericJoiner { +public abstract class JoinFunction extends AbstractRichFunction implements GenericJoiner { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java index 64aec2a4b38e0..9a74133079144 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericMap; /** @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class MapFunction extends AbstractFunction implements GenericMap { +public abstract class MapFunction extends AbstractRichFunction implements GenericMap { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java index aea6bf89d32d2..a4e80945c0b85 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericReduce; /** @@ -44,7 +44,7 @@ * * @param Type of the elements that this function processes. */ -public abstract class ReduceFunction extends AbstractFunction implements GenericReduce { +public abstract class ReduceFunction extends AbstractRichFunction implements GenericReduce { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java index ed4a7866db511..90213d3107ba6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.operators; +import org.apache.flink.api.common.functions.FilterFunctional; import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.functions.FilterFunction; @@ -33,10 +34,10 @@ */ public class FilterOperator extends SingleInputUdfOperator> { - protected final FilterFunction function; + protected final FilterFunctional function; - public FilterOperator(DataSet input, FilterFunction function) { + public FilterOperator(DataSet input, FilterFunctional function) { super(input, input.getType()); this.function = function; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java index 2040a27e78ab3..583f130dc0186 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java @@ -39,7 +39,7 @@ public interface UdfOperator> { /** * Gets the configuration parameters that will be passed to the UDF's open method - * {@link org.apache.flink.api.common.functions.AbstractFunction#open(Configuration)}. + * {@link org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)}. * The configuration is set via the {@link #withParameters(Configuration)} * method. * @@ -69,7 +69,7 @@ public interface UdfOperator> { /** * Sets the configuration parameters for the UDF. These are optional parameters that are passed - * to the UDF in the {@link org.apache.flink.api.common.functions.AbstractFunction#open(Configuration)} method. + * to the UDF in the {@link org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)} method. * * @param parameters The configuration parameters for the UDF. * @return The operator itself, to allow chaining function calls. @@ -83,7 +83,7 @@ public interface UdfOperator> { * {@link org.apache.flink.api.common.functions.RuntimeContext#getBroadcastVariable(String)}. * * The runtime context itself is available in all UDFs via - * {@link org.apache.flink.api.common.functions.AbstractFunction#getRuntimeContext()}. + * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()}. * * @param data The data set to be broadcasted. * @param name The name under which the broadcast data set retrieved. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java index 7fb9c0ffe23f9..37ad02eae3938 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.operators.translation; +import org.apache.flink.api.common.functions.FilterFunctional; import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FilterOperatorBase; @@ -28,7 +29,7 @@ public class PlanFilterOperator extends FilterOperatorBase> { - public PlanFilterOperator(FilterFunction udf, String name, TypeInformation type) { + public PlanFilterOperator(FilterFunctional udf, String name, TypeInformation type) { super(new FlatMapFilter(udf), new UnaryOperatorInformation(type, type), name); } @@ -38,7 +39,7 @@ public static final class FlatMapFilter extends WrappingFunction wrapped) { + private FlatMapFilter(FilterFunctional wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java index 521814c622d7a..25acdfb467226 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericMap; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; @@ -32,7 +32,7 @@ public PlanProjectOperator(int[] fields, String name, TypeInformation inType, } public static final class MapProjector - extends AbstractFunction + extends AbstractRichFunction implements GenericMap { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index c98df6b373777..477e4b37e2ab2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -29,14 +29,14 @@ import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Value; -public abstract class WrappingFunction extends AbstractFunction { +public abstract class WrappingFunction extends AbstractRichFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java index 633adabb69f82..1a3d4789e0094 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -29,7 +29,7 @@ /** * The CoGroupFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CoGroupOperator}. */ -public abstract class CoGroupFunction extends AbstractFunction implements GenericCoGrouper { +public abstract class CoGroupFunction extends AbstractRichFunction implements GenericCoGrouper { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java index b2185a2875a4a..ac5817c7c9951 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.record.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCrosser; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -27,7 +27,7 @@ /** * The CrossFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CrossOperator}. */ -public abstract class CrossFunction extends AbstractFunction implements GenericCrosser { +public abstract class CrossFunction extends AbstractRichFunction implements GenericCrosser { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java index 0222c63babf1a..a37df4cd36d9c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.record.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -28,7 +28,7 @@ * The JoinFunction must implementation by functions of a {@link org.apache.flink.api.java.operators.JoinOperator}. * It resembles an equality join of both inputs on their key fields. */ -public abstract class JoinFunction extends AbstractFunction implements GenericJoiner { +public abstract class JoinFunction extends AbstractRichFunction implements GenericJoiner { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/MapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/MapFunction.java index 88b62820fe11b..99c945ddfd640 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/MapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/MapFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.record.functions; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -28,7 +28,7 @@ * The MapFunction must be extended to provide a mapper implementation * By definition, the mapper is called for each individual input record. */ -public abstract class MapFunction extends AbstractFunction implements GenericCollectorMap { +public abstract class MapFunction extends AbstractRichFunction implements GenericCollectorMap { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java index 4b1dbb3ca0ee0..578442bef2fc5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.types.Record; @@ -31,7 +31,7 @@ * The ReduceFunction must be extended to provide a reducer implementation, as invoked by a * {@link org.apache.flink.api.java.operators.ReduceOperator}. */ -public abstract class ReduceFunction extends AbstractFunction implements GenericGroupReduce, GenericCombine { +public abstract class ReduceFunction extends AbstractRichFunction implements GenericGroupReduce, GenericCombine { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java index 636c492cbefd9..34cd2326c515a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java @@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -54,7 +54,7 @@ /** * The base class for all tasks able to participate in an iteration. */ -public abstract class AbstractIterativePactTask extends RegularPactTask +public abstract class AbstractIterativePactTask extends RegularPactTask implements Terminable { private static final Log log = LogFactory.getLog(AbstractIterativePactTask.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java index d7f3b508a7701..7a77cffe71aba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java @@ -25,7 +25,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -75,7 +75,7 @@ * The type of the feed-back data set (bulk partial solution / workset). For bulk iterations, {@code Y} is the * same as {@code X} */ -public class IterationHeadPactTask extends AbstractIterativePactTask { +public class IterationHeadPactTask extends AbstractIterativePactTask { private static final Log log = LogFactory.getLog(IterationHeadPactTask.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java index 25a614966da7c..2a8325c65ecea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java @@ -23,7 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.runtime.io.network.api.BufferWriter; import org.apache.flink.runtime.io.network.channels.EndOfSuperstepEvent; import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel; @@ -41,7 +41,7 @@ * a {@link BlockingBackChannel} for the workset -XOR- a {@link MutableHashTable} for the solution set. In this case * this task must be scheduled on the same instance as the head. */ -public class IterationIntermediatePactTask extends AbstractIterativePactTask { +public class IterationIntermediatePactTask extends AbstractIterativePactTask { private static final Log log = LogFactory.getLog(IterationIntermediatePactTask.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java index 570630f9acc52..942e2f6e8a02d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier; import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker; import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector; @@ -38,7 +38,7 @@ *

* If there is a separate solution set tail, the iteration head has to make sure to wait for it to finish. */ -public class IterationTailPactTask extends AbstractIterativePactTask +public class IterationTailPactTask extends AbstractIterativePactTask implements PactTaskContext { private static final Log log = LogFactory.getLog(IterationTailPactTask.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java index ffe27e6c28a06..33d8a18e59167 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -28,15 +28,15 @@ * * @param The data type. */ -public class NoOpDriver implements PactDriver { +public class NoOpDriver implements PactDriver { - private PactTaskContext taskContext; + private PactTaskContext taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext context) { + public void setup(PactTaskContext context) { this.taskContext = context; this.running = true; } @@ -47,7 +47,7 @@ public int getNumberOfInputs() { } @Override - public Class getStubType() { + public Class getStubType() { return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java index b515d03d8c7d5..2cb9b445bcbd4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; /** @@ -32,7 +32,7 @@ * @param The type of stub driven by this driver. * @param The data type of the records produced by this driver. */ -public interface PactDriver { +public interface PactDriver { void setup(PactTaskContext context); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 45fe05a0e203d..8c5aa9f21b586 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; @@ -77,7 +77,7 @@ * The abstract base class for all tasks. Encapsulated common behavior and implements the main life-cycle * of the user code. */ -public class RegularPactTask extends AbstractInvokable implements PactTaskContext { +public class RegularPactTask extends AbstractInvokable implements PactTaskContext { protected static final Log LOG = LogFactory.getLog(RegularPactTask.class); @@ -1375,7 +1375,7 @@ public static void initOutputWriters(List writers) { // -------------------------------------------------------------------------------------------- /** - * Opens the given stub using its {@link Function#open(Configuration)} method. If the open call produces + * Opens the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#open(Configuration)} method. If the open call produces * an exception, a new exception with a standard error message is created, using the encountered exception * as its cause. * @@ -1384,7 +1384,7 @@ public static void initOutputWriters(List writers) { * * @throws Exception Thrown, if the user code's open method produces an exception. */ - public static void openUserCode(Function stub, Configuration parameters) throws Exception { + public static void openUserCode(RichFunction stub, Configuration parameters) throws Exception { try { stub.open(parameters); } catch (Throwable t) { @@ -1393,7 +1393,7 @@ public static void openUserCode(Function stub, Configuration parameters) throws } /** - * Closes the given stub using its {@link Function#close()} method. If the close call produces + * Closes the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#close()} method. If the close call produces * an exception, a new exception with a standard error message is created, using the encountered exception * as its cause. * @@ -1401,7 +1401,7 @@ public static void openUserCode(Function stub, Configuration parameters) throws * * @throws Exception Thrown, if the user code's close method produces an exception. */ - public static void closeUserCode(Function stub) throws Exception { + public static void closeUserCode(RichFunction stub) throws Exception { try { stub.close(); } catch (Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java index 6b848d20eba98..e46b2ad551d7d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; /** @@ -32,7 +32,7 @@ * @param The type of stub driven by this driver. * @param The data type of the records produced by this driver. */ -public interface ResettablePactDriver extends PactDriver { +public interface ResettablePactDriver extends PactDriver { boolean isInputResettable(int inputNum); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java index 3f7ad612a9ce8..3dbab78299106 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -61,7 +61,7 @@ public void cancelTask() { // -------------------------------------------------------------------------------------------- - public Function getStub() { + public RichFunction getStub() { return this.mapper; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 8b67041a0bb5e..3610b8d3e53e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -71,7 +71,7 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec public abstract void cancelTask(); - public abstract Function getStub(); + public abstract RichFunction getStub(); public abstract String getTaskName(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java index cca6838fd678e..dc73cb5476b16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -61,7 +61,7 @@ public void cancelTask() { // -------------------------------------------------------------------------------------------- - public Function getStub() { + public RichFunction getStub() { return this.mapper; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java index 3ae324c5efdd1..0362b4a2fb85d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericMap; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -61,7 +61,7 @@ public void cancelTask() { // -------------------------------------------------------------------------------------------- - public Function getStub() { + public RichFunction getStub() { return this.mapper; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedTerminationCriterionDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedTerminationCriterionDriver.java index 76d860bad36ab..3a424691ec93e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedTerminationCriterionDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedTerminationCriterionDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.operators.base.BulkIterationBase; import org.apache.flink.api.common.operators.base.BulkIterationBase.TerminationCriterionAggregator; @@ -47,7 +47,7 @@ public void cancelTask() {} // -------------------------------------------------------------------------------------------- - public Function getStub() { + public RichFunction getStub() { return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java index d5ce0a78aabaf..63778aa59c408 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; @@ -126,7 +126,7 @@ public void cancelTask() { // -------------------------------------------------------------------------------------------- - public Function getStub() { + public RichFunction getStub() { return this.combiner; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java index e72502429a937..c0b8431b0527e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java @@ -35,7 +35,7 @@ import org.apache.flink.api.common.aggregators.AggregatorWithName; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -303,7 +303,7 @@ public void setDriver(@SuppressWarnings("rawtypes") Class this.config.setString(DRIVER_CLASS, driver.getName()); } - public Class> getDriver() { + public Class> getDriver() { final String className = this.config.getString(DRIVER_CLASS, null); if (className == null) { throw new CorruptConfigurationException("The pact driver class is missing."); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java index 303d921a6bc2d..6165a2e212f73 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java @@ -26,7 +26,7 @@ import java.util.Map; import java.util.Map.Entry; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -39,8 +39,6 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.hash.BuildFirstHashMatchIterator; -import org.apache.flink.runtime.operators.hash.BuildSecondHashMatchIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; @@ -695,7 +693,7 @@ public void join(Record rec1, Record rec2, Collector out) } } - static final class RecordIntPairMatchRemovingMatcher extends AbstractFunction implements GenericJoiner + static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements GenericJoiner { private final Map> toRemoveFrom; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index c1a2fd995235a..304d7045c5364 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -24,7 +24,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; @@ -47,7 +47,7 @@ import org.junit.After; import org.junit.BeforeClass; -public class DriverTestBase implements PactTaskContext { +public class DriverTestBase implements PactTaskContext { protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java index faa87b5962b27..779640d6f0a9e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.java.record.io.DelimitedInputFormat; @@ -80,10 +80,10 @@ public Configuration getConfiguration() { return this.mockEnv.getTaskConfiguration(); } - public void registerTask(AbstractInvokable task, @SuppressWarnings("rawtypes") Class driver, Class stubClass) { + public void registerTask(AbstractInvokable task, @SuppressWarnings("rawtypes") Class driver, Class stubClass) { final TaskConfig config = new TaskConfig(this.mockEnv.getTaskConfiguration()); config.setDriver(driver); - config.setStubWrapper(new UserCodeClassWrapper(stubClass)); + config.setStubWrapper(new UserCodeClassWrapper(stubClass)); task.setEnvironment(this.mockEnv); diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala index 759b4446b0377..66d94baa6f4cf 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala @@ -33,7 +33,7 @@ import org.apache.flink.api.scala.analysis.UDF0 import org.apache.flink.api.scala.analysis.FieldSelector import org.apache.flink.configuration.Configuration -import org.apache.flink.api.common.functions.AbstractFunction +import org.apache.flink.api.common.functions.AbstractRichFunction import org.apache.flink.api.java.record.operators.BulkIteration import org.apache.flink.api.common.operators.base.BulkIterationBase import org.apache.flink.api.java.record.operators.DeltaIteration diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java index f450483aaf530..ccdd52e1c8311 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java @@ -25,7 +25,7 @@ import java.util.Map; import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.GenericDataSourceBase; @@ -166,7 +166,7 @@ public T getNode(String name) { } @SuppressWarnings("unchecked") - public T getNode(String name, Class stubClass) { + public T getNode(String name, Class stubClass) { List nodes = this.map.get(name); if (nodes == null || nodes.isEmpty()) { throw new RuntimeException("No node found with the given name and stub class."); diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java index de273901cd456..28bdd010d0800 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java @@ -27,7 +27,7 @@ import java.util.Set; import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.record.operators.BulkIteration; import org.apache.flink.api.java.record.operators.DeltaIteration; @@ -63,7 +63,7 @@ public > T getNode(String name) { } @SuppressWarnings("unchecked") - public > T getNode(String name, Class stubClass) { + public > T getNode(String name, Class stubClass) { List> nodes = this.map.get(name); if (nodes == null || nodes.isEmpty()) { throw new RuntimeException("No node found with the given name and stub class."); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index 0475a4f6d91c5..3d6479ba8ccc4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -22,7 +22,7 @@ import java.util.Iterator; import java.util.Set; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; @@ -32,7 +32,7 @@ import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator; import org.apache.flink.util.Collector; -public class CustomCompensatableDotProductCoGroup extends AbstractFunction implements GenericCoGrouper { +public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements GenericCoGrouper { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java index 28c77ba8ec286..3bd049da4c23f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java @@ -22,7 +22,7 @@ import java.util.Random; import java.util.Set; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; @@ -31,7 +31,7 @@ import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling; import org.apache.flink.util.Collector; -public class CustomCompensatableDotProductMatch extends AbstractFunction implements +public class CustomCompensatableDotProductMatch extends AbstractRichFunction implements GenericJoiner { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java index 74426c0ab19ca..d83b33bd04a4a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java @@ -21,7 +21,7 @@ import java.util.Set; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; @@ -29,7 +29,7 @@ import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStats; import org.apache.flink.util.Collector; -public class CustomCompensatingMap extends AbstractFunction implements GenericCollectorMap { +public class CustomCompensatingMap extends AbstractRichFunction implements GenericCollectorMap { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java index 8af92474b8cec..52f9b3cb22de6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java @@ -20,14 +20,14 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; import org.apache.flink.util.Collector; -public class CustomRankCombiner extends AbstractFunction implements GenericGroupReduce, +public class CustomRankCombiner extends AbstractRichFunction implements GenericGroupReduce, GenericCombine { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java index 15640c0b57eb7..441dc39ab6d66 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java @@ -89,7 +89,7 @@ public static class FilterO extends MapFunction implements Serializable { /** * Reads the filter literals from the configuration. * - * @see org.apache.flink.api.common.functions.Function#open(org.apache.flink.configuration.Configuration) + * @see org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration) */ @Override public void open(Configuration parameters) { From 56b27cc46665fcbc5cd69bfeb19fe695c89d64a3 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 18 Jul 2014 16:48:48 +0200 Subject: [PATCH 03/14] changes to wrapping function --- .../common/functions/FilterFunctional.java | 4 +-- .../flink/api/common/functions/Function.java | 18 ++++++++++ .../api/java/operators/FilterOperator.java | 1 - .../translation/PlanFilterOperator.java | 3 +- .../translation/WrappingFunction.java | 34 +++++++++++++++---- 5 files changed, 49 insertions(+), 11 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java index c5ba78f0cf582..ba0a648c5cfbc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java @@ -18,9 +18,9 @@ package org.apache.flink.api.common.functions; +import java.io.Serializable; - -public interface FilterFunctional extends Function { +public interface FilterFunctional extends Function, Serializable { /** * User defined function for a filter. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java index 00ea0924ba7e5..c2a201f701083 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java @@ -1,3 +1,21 @@ +/** + * 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.common.functions; /** diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java index 90213d3107ba6..9ede5e08d017a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.functions.FilterFunctional; import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.operators.translation.PlanFilterOperator; import org.apache.flink.api.java.DataSet; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java index 37ad02eae3938..347fc68e95cd8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FilterOperatorBase; -import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.types.TypeInformation; import org.apache.flink.util.Collector; @@ -33,7 +32,7 @@ public PlanFilterOperator(FilterFunctional udf, String name, TypeInformation< super(new FlatMapFilter(udf), new UnaryOperatorInformation(type, type), name); } - public static final class FlatMapFilter extends WrappingFunction> + public static final class FlatMapFilter extends WrappingFunction> implements GenericFlatMap { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index 477e4b37e2ab2..3dccd62eea057 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -30,13 +30,15 @@ import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Value; -public abstract class WrappingFunction extends AbstractRichFunction { +public abstract class WrappingFunction extends AbstractRichFunction { private static final long serialVersionUID = 1L; @@ -50,12 +52,12 @@ protected WrappingFunction(T wrappedFunction) { @Override public void open(Configuration parameters) throws Exception { - this.wrappedFunction.open(parameters); + openFunction(this.wrappedFunction, parameters); } @Override public void close() throws Exception { - this.wrappedFunction.close(); + closeFunction(this.wrappedFunction); } @Override @@ -63,13 +65,33 @@ public void setRuntimeContext(RuntimeContext t) { super.setRuntimeContext(t); if (t instanceof IterationRuntimeContext) { - this.wrappedFunction.setRuntimeContext(new WrappingIterationRuntimeContext(t)); + setFunctionRuntimeContext(this.wrappedFunction, new WrappingIterationRuntimeContext(t)); } else{ - this.wrappedFunction.setRuntimeContext(new WrappingRuntimeContext(t)); + setFunctionRuntimeContext(this.wrappedFunction, new WrappingRuntimeContext(t)); + } + } + + private static void openFunction (Function function, Configuration parameters) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.open (parameters); + } + } + + private static void closeFunction (Function function) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.close (); + } + } + + private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.setRuntimeContext(context); } } - private static class WrappingRuntimeContext implements RuntimeContext { From 2eb3aa8a7d301fa5f52c0311ee5cb4fd9920065d Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Sat, 19 Jul 2014 13:22:38 +0200 Subject: [PATCH 04/14] map and reduce operators --- flink-clients/pom.xml | 11 +++ .../compiler/FeedbackPropertiesMatchTest.java | 4 +- .../{GenericMap.java => MapFunctional.java} | 4 +- ...nericReduce.java => ReduceFunctional.java} | 4 +- .../common/operators/AbstractUdfOperator.java | 4 +- .../common/operators/SingleInputOperator.java | 4 +- .../operators/base/MapOperatorBase.java | 4 +- .../operators/base/ReduceOperatorBase.java | 6 +- flink-examples/flink-java-examples/pom.xml | 12 +++ flink-examples/pom.xml | 16 ++++ flink-java/pom.xml | 16 ++++ .../org/apache/flink/api/java/DataSet.java | 6 +- .../flink/api/java/functions/MapFunction.java | 4 +- .../api/java/functions/ReduceFunction.java | 4 +- .../api/java/operators/CoGroupOperator.java | 26 +++---- .../api/java/operators/DistinctOperator.java | 4 +- .../api/java/operators/JoinOperator.java | 26 +++---- .../flink/api/java/operators/MapOperator.java | 10 +-- .../api/java/operators/ProjectOperator.java | 4 +- .../java/operators/ReduceGroupOperator.java | 4 +- .../api/java/operators/ReduceOperator.java | 24 +++--- .../translation/PlanProjectOperator.java | 6 +- .../PlanUnwrappingReduceOperator.java | 13 ++-- .../api/java/typeutils/TypeExtractor.java | 8 +- .../typeutils/runtime/TupleComparator.java | 2 +- .../typeutils/runtime/TupleSerializer.java | 2 +- .../api/java/lambdas/LambdaFunctionsTest.java | 75 +++++++++++++++++++ .../type/extractor/TypeExtractorTest.java | 19 ++++- .../runtime/operators/AllReduceDriver.java | 16 ++-- .../flink/runtime/operators/MapDriver.java | 16 ++-- .../flink/runtime/operators/PactDriver.java | 4 +- .../operators/ReduceCombineDriver.java | 16 ++-- .../flink/runtime/operators/ReduceDriver.java | 16 ++-- .../runtime/operators/RegularPactTask.java | 59 ++++++++++++--- .../operators/chaining/ChainedDriver.java | 4 +- .../operators/chaining/ChainedMapDriver.java | 30 ++++++-- .../runtime/operators/util/TaskConfig.java | 4 +- .../drivers/AllReduceDriverTest.java | 22 +++--- .../drivers/ReduceCombineDriverTest.java | 22 +++--- .../operators/drivers/ReduceDriverTest.java | 22 +++--- 40 files changed, 376 insertions(+), 177 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericMap.java => MapFunctional.java} (91%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericReduce.java => ReduceFunctional.java} (90%) create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index 8944385457338..e0ca970757306 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -154,6 +154,17 @@ under the License. + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java index d50a7d695c392..d0412aa8b6c53 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java @@ -23,7 +23,7 @@ import static org.junit.Assert.*; import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.OperatorInformation; import org.apache.flink.api.common.operators.Order; @@ -1426,7 +1426,7 @@ private static final DataSourceNode getSourceNode() { } private static final MapNode getMapNode() { - return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); + return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); } private static final MatchNode getJoinNode() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java similarity index 91% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java index d022e95fe1b21..873172976f8ff 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericMap.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java @@ -20,7 +20,9 @@ package org.apache.flink.api.common.functions; -public interface GenericMap extends RichFunction { +import java.io.Serializable; + +public interface MapFunctional extends Function, Serializable { /** * A user-implemented function that modifies or transforms an incoming object and diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java similarity index 90% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java index db01d949609f0..4c03c6492f9fd 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericReduce.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java @@ -20,7 +20,9 @@ package org.apache.flink.api.common.functions; -public interface GenericReduce extends RichFunction { +import java.io.Serializable; + +public interface ReduceFunctional extends Function, Serializable { T reduce(T value1, T value2) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java index 15243044fc6ee..055765aa87cf7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java @@ -22,7 +22,7 @@ import java.util.HashMap; import java.util.Map; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** @@ -30,7 +30,7 @@ * * @param Type of the user function */ -public abstract class AbstractUdfOperator extends Operator { +public abstract class AbstractUdfOperator extends Operator { /** * The object or class containing the user function. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java index 9950cf67818a7..38c68f5971356 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.util.Visitor; @@ -32,7 +32,7 @@ * @param Output type of the user function * @param Type of the user function */ -public abstract class SingleInputOperator extends AbstractUdfOperator { +public abstract class SingleInputOperator extends AbstractUdfOperator { /** * The input which produces the data consumed by this operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java index efd8fa94f3ecd..05c039d1c202c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -33,7 +33,7 @@ * @param The result type. * @param The type of the user-defined function. */ -public class MapOperatorBase> extends SingleInputOperator { +public class MapOperatorBase> extends SingleInputOperator { public MapOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java index 62996eaf35be1..05e7accf8fabc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -31,12 +31,12 @@ * Base data flow operator for Reduce user-defined functions. Accepts reduce functions * and key positions. The key positions are expected in the flattened common data model. * - * @see GenericReduce + * @see org.apache.flink.api.common.functions.ReduceFunctional * * @param The type (parameters and return type) of the reduce function. * @param The type of the reduce function. */ -public class ReduceOperatorBase> extends SingleInputOperator { +public class ReduceOperatorBase> extends SingleInputOperator { /** * Creates a grouped reduce data flow operator. diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 549e95b71258c..9fb6513c2aa40 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -318,6 +318,18 @@ under the License. + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 917461a65cd73..34c1c7c3d4a0a 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -51,4 +51,20 @@ under the License. flink-scala-examples + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + + diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 203094391a108..511bfa7eb7132 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -34,6 +34,22 @@ under the License. jar + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + + org.apache.flink diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 44ed8eb7a9e3d..12fb350f3cb5b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -20,6 +20,8 @@ import org.apache.commons.lang3.Validate; import org.apache.flink.api.common.functions.FilterFunctional; +import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.java.aggregation.Aggregations; @@ -135,7 +137,7 @@ public TypeInformation getType() { * @see MapOperator * @see DataSet */ - public MapOperator map(MapFunction mapper) { + public MapOperator map(MapFunctional mapper) { if (mapper == null) { throw new NullPointerException("Map function must not be null."); } @@ -276,7 +278,7 @@ public AggregateOperator min (int field) { * @see ReduceOperator * @see DataSet */ - public ReduceOperator reduce(ReduceFunction reducer) { + public ReduceOperator reduce(ReduceFunctional reducer) { if (reducer == null) { throw new NullPointerException("Reduce function must not be null."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java index 9a74133079144..04a4be0c1170e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; /** * The abstract base class for Map functions. Map functions take elements and transform them, @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class MapFunction extends AbstractRichFunction implements GenericMap { +public abstract class MapFunction extends AbstractRichFunction implements MapFunctional { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java index a4e80945c0b85..2448fb15e3cec 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; /** * The abstract base class for Reduce functions. Reduce functions combine groups of elements to @@ -44,7 +44,7 @@ * * @param Type of the elements that this function processes. */ -public abstract class ReduceFunction extends AbstractRichFunction implements GenericReduce { +public abstract class ReduceFunction extends AbstractRichFunction implements ReduceFunctional { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 8748556da0b99..285316982231b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GenericCoGrouper; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -199,10 +199,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); cogroup.setFirstInput(keyMapper1); @@ -236,10 +236,10 @@ private static PlanUnwrappingCoGroupOperator tr final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -274,10 +274,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index cb7db063b57e9..034da71d95f0f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; @@ -127,7 +127,7 @@ private static PlanUnwrappingReduceGroupOperator transl PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - MapOperatorBase, GenericMap>> mapper = new MapOperatorBase, GenericMap>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, MapFunctional>> mapper = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index 5ca1068e7fd4a..fc388fffccb82 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Operator; @@ -313,10 +313,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); join.setFirstInput(keyMapper1); @@ -350,10 +350,10 @@ private static PlanUnwrappingJoinOperator trans final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -388,10 +388,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, GenericMap>> keyMapper1 = - new MapOperatorBase, GenericMap>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, GenericMap>> keyMapper2 = - new MapOperatorBase, GenericMap>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunctional>> keyMapper1 = + new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunctional>> keyMapper2 = + new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java index 03c6037146cbd..7e3fbcf4bdaca 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; @@ -38,10 +38,10 @@ */ public class MapOperator extends SingleInputUdfOperator> { - protected final MapFunction function; + protected final MapFunctional function; - public MapOperator(DataSet input, MapFunction function) { + public MapOperator(DataSet input, MapFunctional function) { super(input, TypeExtractor.getMapReturnTypes(function, input.getType())); this.function = function; @@ -49,11 +49,11 @@ public MapOperator(DataSet input, MapFunction function) { } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); + MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index 9e9467072f254..a6fed7c23c7d2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; @@ -51,7 +51,7 @@ public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo retur } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection " + Arrays.toString(fields); // create operator PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java index 099860c431302..5e993dc5d8627 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -189,7 +189,7 @@ private static PlanUnwrappingReduceGroupOperator transl PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - MapOperatorBase, GenericMap>> mapper = new MapOperatorBase, GenericMap>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, MapFunctional>> mapper = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java index 12e0f895d24e0..c39ab1e8e5ab8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.GenericMap; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; @@ -44,7 +44,7 @@ */ public class ReduceOperator extends SingleInputUdfOperator> { - private final ReduceFunction function; + private final ReduceFunctional function; private final Grouping grouper; @@ -55,7 +55,7 @@ public class ReduceOperator extends SingleInputUdfOperator input, ReduceFunction function) { + public ReduceOperator(DataSet input, ReduceFunctional function) { super(input, input.getType()); this.function = function; @@ -65,7 +65,7 @@ public ReduceOperator(DataSet input, ReduceFunction function) { } - public ReduceOperator(Grouping input, ReduceFunction function) { + public ReduceOperator(Grouping input, ReduceFunctional function) { super(input.getDataSet(), input.getDataSet().getType()); this.function = function; @@ -83,8 +83,8 @@ public ReduceOperator(Grouping input, ReduceFunction function) { if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, new int[0], name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, new int[0], name); // set input po.setInput(input); @@ -109,8 +109,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // reduce with field positions int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); // set input po.setInput(input); @@ -128,7 +128,7 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // -------------------------------------------------------------------------------------------- private static MapOperatorBase, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys rawKeys, - ReduceFunction function, TypeInformation inputType, String name, Operator input, int dop) + ReduceFunctional function, TypeInformation inputType, String name, Operator input, int dop) { @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys = (Keys.SelectorFunctionKeys) rawKeys; @@ -139,8 +139,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || PlanUnwrappingReduceOperator reducer = new PlanUnwrappingReduceOperator(function, keys, name, inputType, typeInfoWithKey); - MapOperatorBase, GenericMap>> keyExtractingMap = new MapOperatorBase, GenericMap>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); - MapOperatorBase, T, GenericMap, T>> keyRemovingMap = new MapOperatorBase, T, GenericMap, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); + MapOperatorBase, MapFunctional>> keyExtractingMap = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, T, MapFunctional, T>> keyRemovingMap = new MapOperatorBase, T, MapFunctional, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); keyExtractingMap.setInput(input); reducer.setInput(keyExtractingMap); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java index 25acdfb467226..1ddc8d8b8e2a0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java @@ -19,13 +19,13 @@ package org.apache.flink.api.java.operators.translation; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.types.TypeInformation; -public class PlanProjectOperator extends MapOperatorBase> { +public class PlanProjectOperator extends MapOperatorBase> { public PlanProjectOperator(int[] fields, String name, TypeInformation inType, TypeInformation outType) { super(new MapProjector(fields, outType.createSerializer().createInstance()), new UnaryOperatorInformation(inType, outType), name); @@ -33,7 +33,7 @@ public PlanProjectOperator(int[] fields, String name, TypeInformation inType, public static final class MapProjector extends AbstractRichFunction - implements GenericMap + implements MapFunctional { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java index 66aa430a9e848..d407f675107d4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java @@ -18,10 +18,9 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.TypeInformation; @@ -31,21 +30,21 @@ * A reduce operator that takes 2-tuples (key-value pairs), and applies the reduce operation only * on the unwrapped values. */ -public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, GenericReduce>> { +public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, ReduceFunctional>> { - public PlanUnwrappingReduceOperator(ReduceFunction udf, Keys.SelectorFunctionKeys key, String name, + public PlanUnwrappingReduceOperator(ReduceFunctional udf, Keys.SelectorFunctionKeys key, String name, TypeInformation type, TypeInformation> typeInfoWithKey) { super(new ReduceWrapper(udf), new UnaryOperatorInformation, Tuple2>(typeInfoWithKey, typeInfoWithKey), key.computeLogicalKeyPositions(), name); } - public static final class ReduceWrapper extends WrappingFunction> - implements GenericReduce> + public static final class ReduceWrapper extends WrappingFunction> + implements ReduceFunctional> { private static final long serialVersionUID = 1L; - private ReduceWrapper(ReduceFunction wrapped) { + private ReduceWrapper(ReduceFunctional wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index a41874ce2e4eb..7ecbfdd855e1d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -36,7 +36,7 @@ import org.apache.flink.api.common.functions.GenericFlatMap; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.java.functions.InvalidTypesException; import org.apache.flink.api.java.functions.KeySelector; @@ -60,12 +60,12 @@ private TypeExtractor() { // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") - public static TypeInformation getMapReturnTypes(GenericMap mapInterface, TypeInformation inType) { - validateInputType(GenericMap.class, mapInterface.getClass(), 0, inType); + public static TypeInformation getMapReturnTypes(MapFunctional mapInterface, TypeInformation inType) { + validateInputType(MapFunctional.class, mapInterface.getClass(), 0, inType); if(mapInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) mapInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericMap.class, mapInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(MapFunctional.class, mapInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java index 8e0abcb64447d..c786345eb1712 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java @@ -235,7 +235,7 @@ public int compare(T first, T second) { try { for (; i < keyPositions.length; i++) { int keyPos = keyPositions[i]; - int cmp = comparators[i].compare(first.getField(keyPos), second.getField(keyPos)); + int cmp = comparators[i].compare((T)first.getField(keyPos), (T)second.getField(keyPos)); if (cmp != 0) { return cmp; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java index 163d8b2ea8dda..2a68a2a0769f7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java @@ -88,7 +88,7 @@ public T createInstance() { @Override public T copy(T from, T reuse) { for (int i = 0; i < arity; i++) { - Object copy = fieldSerializers[i].copy(from.getField(i), reuse.getField(i)); + Object copy = fieldSerializers[i].copy((T)from.getField(i), (T)reuse.getField(i)); reuse.setField(copy, i); } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java new file mode 100644 index 0000000000000..3d1300c5e3de0 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java @@ -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.java.lambdas; + +import junit.framework.Assert; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.junit.Test; + +public class LambdaFunctionsTest { + private static String TEXT = "Aaaa\n" + + "Baaa\n" + + "Caaa\n" + + "Daaa\n" + + "Eaaa\n" + + "Faaa\n" + + "Gaaa\n" + + "Haaa\n" + + "Hbbb\n"; + + @Test + public void testFilterLambda () { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet text = env.fromElements(TEXT); + + try { + DataSet result = text + .filter(s -> s.startsWith("H")); + result.print(); + Assert.assertNotNull(result); + DataSet result2 = text + .filter(s -> s.startsWith("I")); + } + catch (Exception e) { + Assert.fail(); + } + + } + + @Test + public void testReduceLambda () { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet text = env.fromElements(TEXT); + + try { + DataSet result = text + .reduce ((s, t) -> s.concat(t).substring(0,t.length())); + result.print(); + + } + catch (Exception e) { + Assert.fail(); + } + + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java index b284052ca1f30..2b7705f8ce199 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java @@ -23,7 +23,7 @@ import java.io.IOException; import java.util.Iterator; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.functions.CrossFunction; @@ -1382,8 +1382,8 @@ public Tuple9 mapInterface = new GenericMap() { + public void testFunction() { + MapFunction mapInterface = new MapFunction() { @Override public void setRuntimeContext(RuntimeContext t) { @@ -1414,4 +1414,17 @@ public Boolean map(String record) throws Exception { TypeInformation ti = TypeExtractor.getMapReturnTypes(mapInterface, BasicTypeInfo.STRING_TYPE_INFO); Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, ti); } + + @Test + public void testInterface() { + MapFunctional mapInterface = new MapFunctional() { + @Override + public Boolean map(String record) throws Exception { + return null; + } + }; + + TypeInformation ti = TypeExtractor.getMapReturnTypes(mapInterface, BasicTypeInfo.STRING_TYPE_INFO); + Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, ti); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java index 30bfae3d7fef5..e892984605145 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -35,13 +35,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see GenericReduce + * @see org.apache.flink.api.common.functions.ReduceFunctional */ -public class AllReduceDriver implements PactDriver, T> { +public class AllReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(AllReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -52,7 +52,7 @@ public class AllReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -63,9 +63,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericReduce.class; + final Class> clazz = (Class>) (Class) ReduceFunctional.class; return clazz; } @@ -94,7 +94,7 @@ public void run() throws Exception { LOG.debug(this.taskContext.formatLogString("AllReduce preprocessing done. Running Reducer code.")); } - final GenericReduce stub = this.taskContext.getStub(); + final ReduceFunctional stub = this.taskContext.getStub(); final MutableObjectIterator input = this.input; final TypeSerializer serializer = this.serializer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java index fe1e0c15947d0..213faaad0eada 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -31,20 +31,20 @@ * The MapTask creates an iterator over all key-value pairs of its input and hands that to the map() method * of the MapFunction. * - * @see GenericMap + * @see org.apache.flink.api.common.functions.MapFunctional * * @param The mapper's input data type. * @param The mapper's output data type. */ -public class MapDriver implements PactDriver, OT> { +public class MapDriver implements PactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -55,9 +55,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericMap.class; + final Class> clazz = (Class>) (Class) MapFunctional.class; return clazz; } @@ -75,7 +75,7 @@ public void prepare() { public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); - final GenericMap function = this.taskContext.getStub(); + final MapFunctional function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java index 2cb9b445bcbd4..b515d03d8c7d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; /** @@ -32,7 +32,7 @@ * @param The type of stub driven by this driver. * @param The data type of the records produced by this driver. */ -public interface PactDriver { +public interface PactDriver { void setup(PactTaskContext context); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java index 4d720852b73e4..b9115ad4236b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java @@ -24,7 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -44,7 +44,7 @@ * * @param The data type consumed and produced by the combiner. */ -public class ReduceCombineDriver implements PactDriver, T> { +public class ReduceCombineDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceCombineDriver.class); @@ -52,13 +52,13 @@ public class ReduceCombineDriver implements PactDriver, T> { private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32; - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private TypeSerializer serializer; private TypeComparator comparator; - private GenericReduce reducer; + private ReduceFunctional reducer; private Collector output; @@ -75,7 +75,7 @@ public class ReduceCombineDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -86,9 +86,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericReduce.class; + final Class> clazz = (Class>) (Class) ReduceFunctional.class; return clazz; } @@ -168,7 +168,7 @@ private void sortAndCombine() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final GenericReduce function = this.reducer; + final ReduceFunctional function = this.reducer; final Collector output = this.output; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java index a7e9305c33f98..88c6f383b96bf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -36,13 +36,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see GenericReduce + * @see org.apache.flink.api.common.functions.ReduceFunctional */ -public class ReduceDriver implements PactDriver, T> { +public class ReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -55,7 +55,7 @@ public class ReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -66,9 +66,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericReduce.class; + final Class> clazz = (Class>) (Class) ReduceFunctional.class; return clazz; } @@ -101,7 +101,7 @@ public void run() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final GenericReduce function = this.taskContext.getStub(); + final ReduceFunctional function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 8c5aa9f21b586..07cff37372a49 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -24,8 +24,10 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.distributions.DataDistribution; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -77,7 +79,7 @@ * The abstract base class for all tasks. Encapsulated common behavior and implements the main life-cycle * of the user code. */ -public class RegularPactTask extends AbstractInvokable implements PactTaskContext { +public class RegularPactTask extends AbstractInvokable implements PactTaskContext { protected static final Log LOG = LogFactory.getLog(RegularPactTask.class); @@ -497,7 +499,7 @@ protected void run() throws Exception { if (this.stub != null) { try { Configuration stubConfig = this.config.getStubParameters(); - this.stub.open(stubConfig); + openFunction(this.stub, stubConfig); stubOpen = true; } catch (Throwable t) { @@ -510,7 +512,7 @@ protected void run() throws Exception { // close. We close here such that a regular close throwing an exception marks a task as failed. if (this.running && this.stub != null) { - this.stub.close(); + closeFunction(this.stub); stubOpen = false; } @@ -525,7 +527,9 @@ protected void run() throws Exception { // modify accumulators.ll; if (this.stub != null) { // collect the counters from the stub - Map> accumulators = this.stub.getRuntimeContext().getAllAccumulators(); + // KOSTAS + // !!! Is this.runtimeUdfContext the right thing to return here? !!! + Map> accumulators = getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); RegularPactTask.reportAndClearAccumulators(getEnvironment(), accumulators, this.chainedTasks); } } @@ -533,7 +537,7 @@ protected void run() throws Exception { // close the input, but do not report any exceptions, since we already have another root cause if (stubOpen) { try { - this.stub.close(); + closeFunction(this.stub); } catch (Throwable t) {} } @@ -583,7 +587,7 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - Map> chainedAccumulators = chainedTask.getStub().getRuntimeContext().getAllAccumulators(); + Map> chainedAccumulators = getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators(); AccumulatorHelper.mergeInto(accumulators, chainedAccumulators); } @@ -607,7 +611,7 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - AccumulatorHelper.resetAndClearAccumulators(chainedTask.getStub().getRuntimeContext().getAllAccumulators()); + AccumulatorHelper.resetAndClearAccumulators(getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators()); } } @@ -693,7 +697,7 @@ protected S initStub(Class stubSuperClass) throws Exception { throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + stubSuperClass.getName() + "' as is required."); } - stub.setRuntimeContext(this.runtimeUdfContext); + setFunctionRuntimeContext(stub, this.runtimeUdfContext); return stub; } catch (ClassCastException ccex) { @@ -1384,9 +1388,9 @@ public static void initOutputWriters(List writers) { * * @throws Exception Thrown, if the user code's open method produces an exception. */ - public static void openUserCode(RichFunction stub, Configuration parameters) throws Exception { + public static void openUserCode(Function stub, Configuration parameters) throws Exception { try { - stub.open(parameters); + openFunction(stub, parameters); } catch (Throwable t) { throw new Exception("The user defined 'open(Configuration)' method in " + stub.getClass().toString() + " caused an exception: " + t.getMessage(), t); } @@ -1401,9 +1405,9 @@ public static void openUserCode(RichFunction stub, Configuration parameters) thr * * @throws Exception Thrown, if the user code's close method produces an exception. */ - public static void closeUserCode(RichFunction stub) throws Exception { + public static void closeUserCode(Function stub) throws Exception { try { - stub.close(); + closeFunction(stub); } catch (Throwable t) { throw new Exception("The user defined 'close()' method caused an exception: " + t.getMessage(), t); } @@ -1505,4 +1509,35 @@ private static int[] asArray(List list) { } return a; } + + private static void openFunction (Function function, Configuration parameters) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.open (parameters); + } + } + + private static void closeFunction (Function function) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.close (); + } + } + + private static RuntimeContext getFunctionRuntimeContext (Function function, RuntimeContext defaultContext){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + return richFunction.getRuntimeContext(); + } + else { + return defaultContext; + } + } + + private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.setRuntimeContext(context); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 3610b8d3e53e2..8b67041a0bb5e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -71,7 +71,7 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec public abstract void cancelTask(); - public abstract RichFunction getStub(); + public abstract Function getStub(); public abstract String getTaskName(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java index 0362b4a2fb85d..499a085ca47fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java @@ -19,25 +19,27 @@ package org.apache.flink.runtime.operators.chaining; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.MapFunctional; import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.GenericMap; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.RegularPactTask; public class ChainedMapDriver extends ChainedDriver { - private GenericMap mapper; + private MapFunctional mapper; // -------------------------------------------------------------------------------------------- @Override public void setup(AbstractInvokable parent) { @SuppressWarnings("unchecked") - final GenericMap mapper = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GenericMap.class); + final MapFunctional mapper = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, MapFunctional.class); this.mapper = mapper; - mapper.setRuntimeContext(getUdfRuntimeContext()); + setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); } @Override @@ -54,14 +56,14 @@ public void closeTask() throws Exception { @Override public void cancelTask() { try { - this.mapper.close(); + closeFunction(this.mapper); } catch (Throwable t) { } } // -------------------------------------------------------------------------------------------- - public RichFunction getStub() { + public Function getStub() { return this.mapper; } @@ -84,4 +86,18 @@ public void collect(IT record) { public void close() { this.outputCollector.close(); } + + private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.setRuntimeContext(context); + } + } + + private static void closeFunction (Function function) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.close (); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java index c0b8431b0527e..e72502429a937 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java @@ -35,7 +35,7 @@ import org.apache.flink.api.common.aggregators.AggregatorWithName; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -303,7 +303,7 @@ public void setDriver(@SuppressWarnings("rawtypes") Class this.config.setString(DRIVER_CLASS, driver.getName()); } - public Class> getDriver() { + public Class> getDriver() { final String className = this.config.getString(DRIVER_CLASS, null); if (className == null) { throw new CorruptConfigurationException("The pact driver class is missing."); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java index de01d748cbc61..c0562b6ec6c01 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java @@ -22,7 +22,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -44,8 +44,8 @@ public class AllReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -71,8 +71,8 @@ public void testAllReduceDriverImmutableEmpty() { public void testAllReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -103,8 +103,8 @@ public void testAllReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -145,8 +145,8 @@ public void testAllReduceDriverImmutable() { public void testAllReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -176,8 +176,8 @@ public void testAllReduceDriverMutable() { Assert.assertEquals(78, res.f1.getValue()); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java index 734e1195efa78..79994fd85bcdd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java @@ -22,7 +22,7 @@ import java.util.Collections; import java.util.List; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -44,8 +44,8 @@ public class ReduceCombineDriverTest { @Test public void testImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -81,8 +81,8 @@ public void testImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -112,8 +112,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -153,8 +153,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); @@ -181,8 +181,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java index e8370cfe0ef06..671315baacec6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.GenericReduce; +import org.apache.flink.api.common.functions.ReduceFunctional; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -43,8 +43,8 @@ public class ReduceDriverTest { @Test public void testReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -76,8 +76,8 @@ public void testReduceDriverImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -104,8 +104,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -142,8 +142,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -169,8 +169,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); From 762c33b37f78e82af76690a8b6e188c361b02f15 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 23 Jul 2014 15:39:21 +0200 Subject: [PATCH 05/14] group reduce refactoring --- .../api/common/functions/Combinable.java | 31 ++++++++++ ...{FilterFunctional.java => Filterable.java} | 2 +- ...enericCombine.java => FlatCombinable.java} | 3 +- ...{GenericFlatMap.java => FlatMappable.java} | 4 +- ...icGroupReduce.java => GroupReducible.java} | 3 +- .../{MapFunctional.java => Mappable.java} | 2 +- .../{ReduceFunctional.java => Reducible.java} | 2 +- .../common/functions/util/FunctionUtils.java | 59 +++++++++++++++++++ .../operators/base/FilterOperatorBase.java | 6 +- .../operators/base/FlatMapOperatorBase.java | 6 +- .../operators/base/MapOperatorBase.java | 4 +- .../operators/base/ReduceOperatorBase.java | 6 +- .../common/operators/util/OperatorUtil.java | 4 +- .../operators/util/OperatorUtilTest.java | 4 +- flink-java/pom.xml | 4 +- .../UnsupportedLambdaExpressionException.java | 13 ++++ .../translation/DoubleWrappingFunction.java | 5 ++ 17 files changed, 135 insertions(+), 23 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java rename flink-core/src/main/java/org/apache/flink/api/common/functions/{FilterFunctional.java => Filterable.java} (94%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericCombine.java => FlatCombinable.java} (91%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericFlatMap.java => FlatMappable.java} (92%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericGroupReduce.java => GroupReducible.java} (93%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{MapFunctional.java => Mappable.java} (93%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{ReduceFunctional.java => Reducible.java} (93%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java new file mode 100644 index 0000000000000..ec89a61edd3c0 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java @@ -0,0 +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.common.functions; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * Generic interface used for combiners. + */ +public interface Combinable extends Function, Serializable { + + T combine(Iterator records) throws Exception; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java similarity index 94% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java index ba0a648c5cfbc..ed5f9a15a1def 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunctional.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java @@ -20,7 +20,7 @@ package org.apache.flink.api.common.functions; import java.io.Serializable; -public interface FilterFunctional extends Function, Serializable { +public interface Filterable extends Function, Serializable { /** * User defined function for a filter. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java similarity index 91% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java index 03d028b5bea4d..05c07ae03189f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.functions; +import java.io.Serializable; import java.util.Iterator; import org.apache.flink.util.Collector; @@ -26,7 +27,7 @@ /** * Generic interface used for combiners. */ -public interface GenericCombine extends RichFunction { +public interface FlatCombinable extends Function, Serializable { void combine(Iterator records, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java similarity index 92% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java index 603da1cdb1b07..19c789880ba23 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericFlatMap.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java @@ -21,13 +21,15 @@ import org.apache.flink.util.Collector; +import java.io.Serializable; + /** * * @param * @param */ -public interface GenericFlatMap extends RichFunction { +public interface FlatMappable extends Function, Serializable { /** * User defined function to perform transformations on records. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java similarity index 93% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java index fe616a4605ad3..44f1bc8b0d7a9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.functions; +import java.io.Serializable; import java.util.Iterator; import org.apache.flink.util.Collector; @@ -29,7 +30,7 @@ * @param Incoming types * @param Outgoing types */ -public interface GenericGroupReduce extends RichFunction { +public interface GroupReducible extends Function, Serializable { /** * * The central function to be implemented for a reducer. The function receives per call one diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java similarity index 93% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java index 873172976f8ff..88d0c7f3f82c7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunctional.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java @@ -22,7 +22,7 @@ import java.io.Serializable; -public interface MapFunctional extends Function, Serializable { +public interface Mappable extends Function, Serializable { /** * A user-implemented function that modifies or transforms an incoming object and diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java similarity index 93% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java index 4c03c6492f9fd..3d87511d5b8e1 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunctional.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java @@ -22,7 +22,7 @@ import java.io.Serializable; -public interface ReduceFunctional extends Function, Serializable { +public interface Reducible extends Function, Serializable { T reduce(T value1, T value2) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java new file mode 100644 index 0000000000000..4439d3538194e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java @@ -0,0 +1,59 @@ +/** + * 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.common.functions.util; + + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; + +public class FunctionUtils { + + public static void openFunction (Function function, Configuration parameters) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.open (parameters); + } + } + + public static void closeFunction (Function function) throws Exception{ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.close (); + } + } + + public static void setFunctionRuntimeContext (Function function, RuntimeContext context){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + richFunction.setRuntimeContext(context); + } + } + + public static RuntimeContext getFunctionRuntimeContext (Function function, RuntimeContext defaultContext){ + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + return richFunction.getRuntimeContext(); + } + else { + return defaultContext; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java index 34896a234de46..8dfe215ce42bc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see GenericFlatMap + * @see org.apache.flink.api.common.functions.FlatMappable */ -public class FilterOperatorBase> extends SingleInputOperator { +public class FilterOperatorBase> extends SingleInputOperator { public FilterOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java index 0de236ef1ef1c..8c1f96b463a59 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see GenericFlatMap + * @see org.apache.flink.api.common.functions.FlatMappable */ -public class FlatMapOperatorBase> extends SingleInputOperator { +public class FlatMapOperatorBase> extends SingleInputOperator { public FlatMapOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java index 05c039d1c202c..92be186f2f99b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -33,7 +33,7 @@ * @param The result type. * @param The type of the user-defined function. */ -public class MapOperatorBase> extends SingleInputOperator { +public class MapOperatorBase> extends SingleInputOperator { public MapOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java index 05e7accf8fabc..f82e6c42d1cb5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -31,12 +31,12 @@ * Base data flow operator for Reduce user-defined functions. Accepts reduce functions * and key positions. The key positions are expected in the flattened common data model. * - * @see org.apache.flink.api.common.functions.ReduceFunctional + * @see org.apache.flink.api.common.functions.Reducible * * @param The type (parameters and return type) of the reduce function. * @param The type of the reduce function. */ -public class ReduceOperatorBase> extends SingleInputOperator { +public class ReduceOperatorBase> extends SingleInputOperator { /** * Creates a grouped reduce data flow operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java index 18097c9afba84..530c23e771ae9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java @@ -28,7 +28,7 @@ import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; @@ -56,7 +56,7 @@ public class OperatorUtil { static { STUB_CONTRACTS.put(GenericCollectorMap.class, CollectorMapOperatorBase.class); - STUB_CONTRACTS.put(GenericGroupReduce.class, GroupReduceOperatorBase.class); + STUB_CONTRACTS.put(GroupReducible.class, GroupReduceOperatorBase.class); STUB_CONTRACTS.put(GenericCoGrouper.class, CoGroupOperatorBase.class); STUB_CONTRACTS.put(GenericCrosser.class, CrossOperatorBase.class); STUB_CONTRACTS.put(GenericJoiner.class, JoinOperatorBase.class); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java index 446f4e3d19fc3..d897504f3f872 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java @@ -21,11 +21,11 @@ import static org.junit.Assert.assertEquals; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.api.common.io.DelimitedInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; @@ -123,5 +123,5 @@ static abstract class Mapper implements GenericCollectorMap static abstract class Matcher implements GenericJoiner {} - static abstract class Reducer implements GenericGroupReduce {} + static abstract class Reducer implements GroupReducible {} } diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 511bfa7eb7132..57e58933f8537 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -42,8 +42,8 @@ under the License. maven-compiler-plugin 3.1 - 1.8 - 1.8 + 1.6 + 1.6 diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java new file mode 100644 index 0000000000000..7eb6e0037a90b --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java @@ -0,0 +1,13 @@ +package org.apache.flink.api.java.functions; + +/** + * Created by kostas on 20/07/14. + */ +public class UnsupportedLambdaExpressionException extends RuntimeException { + + private static final long serialVersionUID = -1721898801986321010L; + + public UnsupportedLambdaExpressionException(String message) { + super(message); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java new file mode 100644 index 0000000000000..6a52f48265a8e --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java @@ -0,0 +1,5 @@ +package org.apache.flink.api.java.operators.translation; + + +public class DoubleWrappingFunction { +} From d041d15cc16c16ea026d7b73673b0a592ba545d7 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Thu, 24 Jul 2014 11:34:41 +0200 Subject: [PATCH 06/14] group reduce as interface -- shallow implementation --- .../org/apache/flink/api/common/functions/Combinable.java | 4 +++- .../{FlatCombinable.java => ProposedNewCombinable.java} | 6 ++---- .../java/operators/translation/DoubleWrappingFunction.java | 5 ----- 3 files changed, 5 insertions(+), 10 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{FlatCombinable.java => ProposedNewCombinable.java} (84%) delete mode 100644 flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java index ec89a61edd3c0..b985d945c7c0d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java @@ -22,10 +22,12 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.flink.util.Collector; + /** * Generic interface used for combiners. */ public interface Combinable extends Function, Serializable { - T combine(Iterator records) throws Exception; + void combine(Iterator records, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java similarity index 84% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java index 05c07ae03189f..92389eb796250 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java @@ -22,12 +22,10 @@ import java.io.Serializable; import java.util.Iterator; -import org.apache.flink.util.Collector; - /** * Generic interface used for combiners. */ -public interface FlatCombinable extends Function, Serializable { +public interface ProposedNewCombinable extends Function, Serializable { - void combine(Iterator records, Collector out) throws Exception; + T combine(Iterator records) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java deleted file mode 100644 index 6a52f48265a8e..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/DoubleWrappingFunction.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.flink.api.java.operators.translation; - - -public class DoubleWrappingFunction { -} From aed25492c994c200a50ff4972abb274fc0387c4e Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Tue, 29 Jul 2014 18:34:26 +0200 Subject: [PATCH 07/14] Java and record API based on SAM interfaces --- .../compiler/FeedbackPropertiesMatchTest.java | 10 +- .../compiler/GroupReduceCompilationTest.java | 8 +- ...nction.java => DummyFlatJoinFunction.java} | 9 +- .../flink/compiler/util/DummyCrossStub.java | 9 +- ...GenericCoGrouper.java => CoGroupable.java} | 9 +- .../{GenericCrosser.java => Crossable.java} | 15 +- .../{Combinable.java => FlatCombinable.java} | 4 +- .../{GenericJoiner.java => FlatJoinable.java} | 8 +- .../api/common/functions/FlatMappable.java | 19 +- .../api/common/functions/GroupReducible.java | 2 +- .../flink/api/common/functions/Joinable.java | 28 + .../flink/api/common/functions/Mappable.java | 14 +- .../flink/api/common/functions/Reducible.java | 13 +- .../common/functions/util/FunctionUtils.java | 12 +- .../common/operators/DualInputOperator.java | 4 +- .../operators/base/CoGroupOperatorBase.java | 6 +- .../operators/base/CrossOperatorBase.java | 6 +- .../base/GroupReduceOperatorBase.java | 17 +- .../operators/base/JoinOperatorBase.java | 6 +- .../common/operators/util/OperatorUtil.java | 12 +- .../operators/util/OperatorUtilTest.java | 12 +- .../java/graph/ConnectedComponents.java | 14 +- .../org/apache/flink/api/java/DataSet.java | 60 +- .../api/java/functions/CoGroupFunction.java | 4 +- .../api/java/functions/CombineFunction.java | 34 + .../api/java/functions/CrossFunction.java | 17 +- .../api/java/functions/FilterFunction.java | 4 +- .../api/java/functions/FlatJoinFunction.java | 75 +++ .../api/java/functions/FlatMapFunction.java | 4 +- .../java/functions/FunctionAnnotation.java | 10 +- .../java/functions/GroupReduceFunction.java | 8 +- .../api/java/functions/JoinFunction.java | 70 +-- .../flink/api/java/functions/MapFunction.java | 4 +- .../api/java/functions/ReduceFunction.java | 4 +- .../UnsupportedLambdaExpressionException.java | 25 +- .../api/java/operators/AggregateOperator.java | 12 +- .../api/java/operators/CoGroupOperator.java | 49 +- .../api/java/operators/CrossOperator.java | 26 +- .../api/java/operators/DistinctOperator.java | 40 +- .../api/java/operators/FilterOperator.java | 10 +- .../api/java/operators/FlatMapOperator.java | 11 +- ...Operator.java => GroupReduceOperator.java} | 57 +- .../api/java/operators/JoinOperator.java | 167 +++-- .../flink/api/java/operators/MapOperator.java | 14 +- .../api/java/operators/ProjectOperator.java | 4 +- .../api/java/operators/ReduceOperator.java | 27 +- .../api/java/operators/SortedGrouping.java | 13 +- .../api/java/operators/UnsortedGrouping.java | 16 +- .../translation/PlanFilterOperator.java | 14 +- .../translation/PlanProjectOperator.java | 6 +- .../PlanUnwrappingCoGroupOperator.java | 17 +- .../PlanUnwrappingJoinOperator.java | 28 +- .../PlanUnwrappingReduceGroupOperator.java | 25 +- .../PlanUnwrappingReduceOperator.java | 12 +- .../translation/WrappingFunction.java | 32 +- .../record/functions/CoGroupFunction.java | 4 +- .../java/record/functions/CrossFunction.java | 18 +- .../java/record/functions/JoinFunction.java | 6 +- .../java/record/functions/ReduceFunction.java | 6 +- .../api/java/typeutils/TypeExtractor.java | 66 +- .../typeutils/runtime/TupleSerializer.java | 2 +- .../api/java/lambdas/LambdaFunctionsTest.java | 75 --- .../DeltaIterationTranslationTest.java | 2 +- .../type/extractor/TypeExtractorTest.java | 13 +- .../AbstractCachedBuildSideMatchDriver.java | 6 +- .../operators/AllGroupReduceDriver.java | 24 +- .../runtime/operators/AllReduceDriver.java | 16 +- .../runtime/operators/CoGroupDriver.java | 14 +- .../CoGroupWithSolutionSetFirstDriver.java | 14 +- .../CoGroupWithSolutionSetSecondDriver.java | 14 +- .../flink/runtime/operators/CrossDriver.java | 32 +- .../runtime/operators/FlatMapDriver.java | 16 +- .../operators/GroupReduceCombineDriver.java | 14 +- .../runtime/operators/GroupReduceDriver.java | 16 +- .../JoinWithSolutionSetFirstDriver.java | 14 +- .../JoinWithSolutionSetSecondDriver.java | 14 +- .../flink/runtime/operators/MapDriver.java | 16 +- .../flink/runtime/operators/MatchDriver.java | 16 +- .../operators/ReduceCombineDriver.java | 16 +- .../flink/runtime/operators/ReduceDriver.java | 16 +- .../runtime/operators/RegularPactTask.java | 60 +- .../operators/ResettablePactDriver.java | 4 +- .../chaining/ChainedFlatMapDriver.java | 21 +- .../operators/chaining/ChainedMapDriver.java | 28 +- .../SynchronousChainedCombineDriver.java | 17 +- .../hash/BuildFirstHashMatchIterator.java | 4 +- .../hash/BuildSecondHashMatchIterator.java | 4 +- .../sort/CombiningUnilateralSortMerger.java | 19 +- .../operators/sort/MergeMatchIterator.java | 14 +- .../operators/util/JoinTaskIterator.java | 4 +- .../operators/CachedMatchTaskTest.java | 4 +- .../operators/CoGroupTaskExternalITCase.java | 6 +- .../runtime/operators/CoGroupTaskTest.java | 6 +- .../operators/CombineTaskExternalITCase.java | 6 +- .../runtime/operators/CombineTaskTest.java | 6 +- .../operators/CrossTaskExternalITCase.java | 6 +- .../runtime/operators/CrossTaskTest.java | 16 +- .../operators/MatchTaskExternalITCase.java | 6 +- .../runtime/operators/MatchTaskTest.java | 14 +- .../operators/ReduceTaskExternalITCase.java | 6 +- .../runtime/operators/ReduceTaskTest.java | 6 +- .../drivers/AllGroupReduceDriverTest.java | 14 +- .../drivers/AllReduceDriverTest.java | 22 +- .../drivers/GroupReduceDriverTest.java | 14 +- .../drivers/ReduceCombineDriverTest.java | 22 +- .../operators/drivers/ReduceDriverTest.java | 22 +- .../hash/HashMatchIteratorITCase.java | 14 +- .../sort/SortMergeMatchIteratorITCase.java | 4 +- .../operators/testutils/DriverTestBase.java | 10 +- .../util/HashVsSortMiniBenchmark.java | 4 +- .../api/scala/functions/CrossFunction.scala | 28 +- .../api/scala/operators/CrossOperator.scala | 69 +- flink-tests/pom.xml | 12 + .../CustomCompensatableDotProductCoGroup.java | 4 +- .../CustomCompensatableDotProductMatch.java | 4 +- .../CustomRankCombiner.java | 8 +- .../test/javaApiOperators/CrossITCase.java | 37 +- .../test/javaApiOperators/FilterITCase.java | 21 +- .../test/javaApiOperators/FlatMapITCase.java | 42 +- .../javaApiOperators/GroupReduceITCase.java | 587 ++++++++++-------- .../test/javaApiOperators/JoinITCase.java | 97 +-- .../test/javaApiOperators/MapITCase.java | 36 +- .../test/javaApiOperators/ReduceITCase.java | 34 +- .../flink/test/operators/CrossITCase.java | 49 +- .../kmeans/udfs/ComputeDistance.java | 16 +- 125 files changed, 1649 insertions(+), 1364 deletions(-) rename flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/{DummyJoinFunction.java => DummyFlatJoinFunction.java} (78%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericCoGrouper.java => CoGroupable.java} (76%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericCrosser.java => Crossable.java} (76%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Combinable.java => FlatCombinable.java} (87%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GenericJoiner.java => FlatJoinable.java} (83%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java rename flink-java/src/main/java/org/apache/flink/api/java/operators/{ReduceGroupOperator.java => GroupReduceOperator.java} (81%) delete mode 100644 flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java index d0412aa8b6c53..b925ff2524790 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java @@ -22,8 +22,8 @@ import static org.apache.flink.compiler.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport.*; import static org.junit.Assert.*; -import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.OperatorInformation; import org.apache.flink.api.common.operators.Order; @@ -47,7 +47,7 @@ import org.apache.flink.compiler.plan.SingleInputPlanNode; import org.apache.flink.compiler.plan.SourcePlanNode; import org.apache.flink.compiler.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport; -import org.apache.flink.compiler.testfunctions.DummyJoinFunction; +import org.apache.flink.compiler.testfunctions.DummyFlatJoinFunction; import org.apache.flink.compiler.testfunctions.IdentityMapper; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.operators.DriverStrategy; @@ -1426,10 +1426,10 @@ private static final DataSourceNode getSourceNode() { } private static final MapNode getMapNode() { - return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); + return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); } private static final MatchNode getJoinNode() { - return new MatchNode(new JoinOperatorBase>(new DummyJoinFunction(), new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op")); + return new MatchNode(new JoinOperatorBase>(new DummyFlatJoinFunction(), new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op")); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java index f3e513a97baf3..8a76471c8401d 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.operators.util.FieldList; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.ReduceGroupOperator; +import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.util.Collector; @@ -94,7 +94,7 @@ public void testAllReduceWithCombiner() { DataSet data = env.generateSequence(1, 8000000).name("source"); - ReduceGroupOperator reduced = data.reduceGroup(new GroupReduceFunction() { + GroupReduceOperator reduced = data.reduceGroup(new GroupReduceFunction() { public void reduce(Iterator values, Collector out) {} }).name("reducer"); @@ -194,7 +194,7 @@ public void testGroupedReduceWithFieldPositionKeyCombinable() { DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class) .name("source").setParallelism(6); - ReduceGroupOperator, Tuple2> reduced = data + GroupReduceOperator, Tuple2> reduced = data .groupBy(1) .reduceGroup(new GroupReduceFunction, Tuple2>() { public void reduce(Iterator> values, Collector> out) {} @@ -309,7 +309,7 @@ public void testGroupedReduceWithSelectorFunctionKeyCombinable() { DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class) .name("source").setParallelism(6); - ReduceGroupOperator, Tuple2> reduced = data + GroupReduceOperator, Tuple2> reduced = data .groupBy(new KeySelector, String>() { public String getKey(Tuple2 value) { return value.f0; } }) diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyJoinFunction.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java similarity index 78% rename from flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyJoinFunction.java rename to flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java index 0db075ff94b77..0211cab6991aa 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyJoinFunction.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java @@ -19,14 +19,15 @@ package org.apache.flink.compiler.testfunctions; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.FlatJoinFunction; +import org.apache.flink.util.Collector; -public class DummyJoinFunction extends JoinFunction { +public class DummyFlatJoinFunction extends FlatJoinFunction { private static final long serialVersionUID = 1L; @Override - public T join(T first, T second) { - return null; + public void join(T first, T second, Collector out) { + out.collect(null); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCrossStub.java b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCrossStub.java index 51ad75d122522..736ee14d2ce80 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCrossStub.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCrossStub.java @@ -23,14 +23,13 @@ import org.apache.flink.api.java.record.functions.CrossFunction; import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; -public class DummyCrossStub extends CrossFunction implements Serializable { +public class DummyCrossStub extends CrossFunction { private static final long serialVersionUID = 1L; + @Override - public void cross(Record record1, Record record2, Collector out) { - out.collect(record1); - out.collect(record2); + public Record cross(Record first, Record second) throws Exception { + return first; } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java similarity index 76% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java index da4ccb5e0493e..29f28c361ec18 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java @@ -19,22 +19,23 @@ package org.apache.flink.api.common.functions; +import java.io.Serializable; import java.util.Iterator; import org.apache.flink.util.Collector; -public interface GenericCoGrouper extends RichFunction { +public interface CoGroupable extends Function, Serializable { /** * This method must be implemented to provide a user implementation of a * coGroup. It is called for each two key-value pairs that share the same * key and come from different inputs. * - * @param records1 The records from the first input which were paired with the key. - * @param records2 The records from the second input which were paired with the key. + * @param first The records from the first input which were paired with the key. + * @param second The records from the second input which were paired with the key. * @param out A collector that collects all output pairs. */ - void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; + void coGroup(Iterator first, Iterator second, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java similarity index 76% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java index e4663479612e1..2b56987e30c62 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCrosser.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java @@ -19,23 +19,24 @@ package org.apache.flink.api.common.functions; -import org.apache.flink.util.Collector; +import java.io.Serializable; /** - * @param First input type - * @param Second input type - * @param Output type + * @param First input type + * @param Second input type + * @param Output type */ -public interface GenericCrosser extends RichFunction { +public interface Crossable extends Function, Serializable { /** * User defined function for the cross operator. * * @param record1 Record from first input * @param record2 Record from the second input - * @param out Collector to submit resulting records. + * @return result of cross UDF. * @throws Exception */ - void cross(V1 record1, V2 record2, Collector out) throws Exception; + OUT cross(IN1 record1, IN2 record2) throws Exception; + } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java similarity index 87% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java index b985d945c7c0d..bb600207d00cd 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Combinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java @@ -27,7 +27,7 @@ /** * Generic interface used for combiners. */ -public interface Combinable extends Function, Serializable { +public interface FlatCombinable extends Function, Serializable { - void combine(Iterator records, Collector out) throws Exception; + void combine(Iterator values, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java similarity index 83% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java index bfa6a278bf04e..b5afb90cbed46 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericJoiner.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java @@ -21,8 +21,10 @@ import org.apache.flink.util.Collector; +import java.io.Serializable; -public interface GenericJoiner extends RichFunction { - - void join(V1 value1, V2 value2, Collector out) throws Exception; + +public interface FlatJoinable extends Function, Serializable { + + void join (IN1 left, IN2 right, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java index 19c789880ba23..c8721e2e5ce77 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java @@ -30,15 +30,16 @@ * @param */ public interface FlatMappable extends Function, Serializable { - + /** - * User defined function to perform transformations on records. - * This method allows to submit an arbitrary number of records - * per incoming tuple. - * - * @param record incoming record - * @param out outgoing collector to return none, one or more records - * @throws Exception + * The core method of FlatMappable. Takes an element from the input data set and transforms + * it into zero, one, or more elements. + * + * @param value The input value. + * @param out The collector for for emitting result values. + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. */ - void flatMap(T record, Collector out) throws Exception; + void flatMap(T value, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java index 44f1bc8b0d7a9..0999c5ba2889c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java @@ -41,5 +41,5 @@ public interface GroupReducible extends Function, Serializable { * @param out The collector to hand results to. * @throws Exception */ - void reduce(Iterator records, Collector out) throws Exception; + void reduce(Iterator values, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java new file mode 100644 index 0000000000000..a810386fa9df2 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java @@ -0,0 +1,28 @@ +/** + * 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.common.functions; + +import java.io.Serializable; + + +public interface Joinable extends Function, Serializable { + + OUT join(IN1 first, IN2 second) throws Exception; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java index 88d0c7f3f82c7..bc80b76122152 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java @@ -23,10 +23,16 @@ import java.io.Serializable; public interface Mappable extends Function, Serializable { - + /** - * A user-implemented function that modifies or transforms an incoming object and - * returns the result. + * The core method of Mappable. Takes an element from the input data set and transforms + * it into exactly one element. + * + * @param value The input value. + * @returns The transformed value + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. */ - O map(T record) throws Exception; + O map(T value) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java index 3d87511d5b8e1..bcd5f6147ed3a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java @@ -23,6 +23,17 @@ import java.io.Serializable; public interface Reducible extends Function, Serializable { - + + /** + * The core method of Reducible, combining two values into one value of the same type. + * The reduce function is consecutively applied to all values of a group until only a single value remains. + * + * @param value1 The first value to combine. + * @param value2 The second value to combine. + * @return The combined value of both input values. + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. + */ T reduce(T value1, T value2) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java index 4439d3538194e..1b14ef6ee5dee 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java @@ -24,8 +24,13 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; +import java.util.regex.Pattern; + public class FunctionUtils { + private static final Pattern lambdaPattern = Pattern.compile("(\\S+)\\$\\$Lambda\\$(\\d+)/\\d+"); + + public static void openFunction (Function function, Configuration parameters) throws Exception{ if (function instanceof RichFunction) { RichFunction richFunction = (RichFunction) function; @@ -47,7 +52,7 @@ public static void setFunctionRuntimeContext (Function function, RuntimeContext } } - public static RuntimeContext getFunctionRuntimeContext (Function function, RuntimeContext defaultContext){ + public static RuntimeContext getFunctionRuntimeContext (Function function, RuntimeContext defaultContext){ if (function instanceof RichFunction) { RichFunction richFunction = (RichFunction) function; return richFunction.getRuntimeContext(); @@ -56,4 +61,9 @@ public static RuntimeContext getFunctionRuntimeContext (Function function, Runt return defaultContext; } } + + public static boolean isLambdaFunction (Function function) { + + return lambdaPattern.matcher(function.getClass().getName()).matches(); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java index 82b6b3064f395..e57c90d230271 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.util.Visitor; @@ -33,7 +33,7 @@ * @param Output type of the user function * @param Type of the user function */ -public abstract class DualInputOperator extends AbstractUdfOperator { +public abstract class DualInputOperator extends AbstractUdfOperator { /** * The operator producing the first input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java index a9ae97cc67b3c..a2414c583ac2f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.Ordering; @@ -28,9 +28,9 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** - * @see GenericCoGrouper + * @see org.apache.flink.api.common.functions.CoGroupable */ -public class CoGroupOperatorBase> extends DualInputOperator { +public class CoGroupOperatorBase> extends DualInputOperator { /** * The ordering for the order inside a group from input one. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java index 33e150d3cb035..fbea862f0633e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericCrosser; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see GenericCrosser + * @see org.apache.flink.api.common.functions.Crossable */ -public class CrossOperatorBase> extends DualInputOperator { +public class CrossOperatorBase> extends DualInputOperator { public CrossOperatorBase(UserCodeWrapper udf, BinaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java index a24826a167e9a..13aa684621128 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java @@ -19,8 +19,9 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; + +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -30,9 +31,9 @@ /** - * @see GenericGroupReduce + * @see org.apache.flink.api.common.functions.GroupReducible */ -public class GroupReduceOperatorBase> extends SingleInputOperator { +public class GroupReduceOperatorBase> extends SingleInputOperator { /** * The ordering for the order inside a reduce group. @@ -91,15 +92,15 @@ public Ordering getGroupOrder() { /** * Marks the group reduce operation as combinable. Combinable operations may pre-reduce the * data before the actual group reduce operations. Combinable user-defined functions - * must implement the interface {@link GenericCombine}. + * must implement the interface {@link org.apache.flink.api.common.functions.FlatCombinable}. * * @param combinable Flag to mark the group reduce operation as combinable. */ public void setCombinable(boolean combinable) { // sanity check - if (combinable && !GenericCombine.class.isAssignableFrom(this.userFunction.getUserCodeClass())) { - throw new IllegalArgumentException("Cannot set a UDF as combinable if it does not implement the interface " + - GenericCombine.class.getName()); + if (combinable && !FlatCombinable.class.isAssignableFrom(this.userFunction.getUserCodeClass())) { + throw new IllegalArgumentException("Cannot set a UDF as combinable if it does not implement the interface " + + FlatCombinable.class.getName()); } else { this.combinable = combinable; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java index b4eeeaa0d8eb0..3fbdefa257e60 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -27,9 +27,9 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** - * @see GenericJoiner + * @see org.apache.flink.api.common.functions.FlatJoinable */ -public class JoinOperatorBase> extends DualInputOperator +public class JoinOperatorBase> extends DualInputOperator { public JoinOperatorBase(UserCodeWrapper udf, BinaryOperatorInformation operatorInfo, int[] keyPositions1, int[] keyPositions2, String name) { super(udf, operatorInfo, keyPositions1, keyPositions2, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java index 530c23e771ae9..b420566c3bdc3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java @@ -25,11 +25,11 @@ import java.util.Map; import java.util.Map.Entry; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.functions.GenericCollectorMap; -import org.apache.flink.api.common.functions.GenericCrosser; import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.GenericJoiner; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.InputFormat; @@ -57,9 +57,9 @@ public class OperatorUtil { static { STUB_CONTRACTS.put(GenericCollectorMap.class, CollectorMapOperatorBase.class); STUB_CONTRACTS.put(GroupReducible.class, GroupReduceOperatorBase.class); - STUB_CONTRACTS.put(GenericCoGrouper.class, CoGroupOperatorBase.class); - STUB_CONTRACTS.put(GenericCrosser.class, CrossOperatorBase.class); - STUB_CONTRACTS.put(GenericJoiner.class, JoinOperatorBase.class); + STUB_CONTRACTS.put(CoGroupable.class, CoGroupOperatorBase.class); + STUB_CONTRACTS.put(Crossable.class, CrossOperatorBase.class); + STUB_CONTRACTS.put(FlatJoinable.class, JoinOperatorBase.class); STUB_CONTRACTS.put(FileInputFormat.class, GenericDataSourceBase.class); STUB_CONTRACTS.put(FileOutputFormat.class, GenericDataSinkBase.class); STUB_CONTRACTS.put(InputFormat.class, GenericDataSourceBase.class); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java index d897504f3f872..38688b6aa3088 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java @@ -21,12 +21,12 @@ import static org.junit.Assert.assertEquals; +import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.functions.GenericCollectorMap; -import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.api.common.io.DelimitedInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; @@ -115,13 +115,13 @@ public void getContractClassShouldReturnSourceForInputFormat() { assertEquals(GenericDataSourceBase.class, result); } - static abstract class CoGrouper implements GenericCoGrouper {} + static abstract class CoGrouper implements CoGroupable {} - static abstract class Crosser implements GenericCrosser {} + static abstract class Crosser implements Crossable {} static abstract class Mapper implements GenericCollectorMap {} - static abstract class Matcher implements GenericJoiner {} + static abstract class Matcher implements FlatJoinable {} static abstract class Reducer implements GroupReducible {} } diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java index b71347a9a4f1d..bf96474a81f8d 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java @@ -22,11 +22,12 @@ import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.FlatJoinFunction; import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; +import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; @@ -165,11 +166,10 @@ public Tuple2 join(Tuple2 vertexWithComponent, Tuple2, Tuple2, Tuple2> { + public static final class ComponentIdFilter extends FlatJoinFunction, Tuple2, Tuple2> { @Override public void join(Tuple2 candidate, Tuple2 old, Collector> out) { @@ -177,10 +177,10 @@ public void join(Tuple2 candidate, Tuple2 old, Collector out.collect(candidate); } } - @Override - public Tuple2 join(Tuple2 first, Tuple2 second) { return null; } } + + @Override public String getDescription() { return "Parameters: "; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 12fb350f3cb5b..092c48cb120f7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -19,9 +19,12 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.Validate; -import org.apache.flink.api.common.functions.FilterFunctional; -import org.apache.flink.api.common.functions.MapFunctional; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Filterable; +import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.java.aggregation.Aggregations; @@ -32,32 +35,29 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.io.CsvOutputFormat; import org.apache.flink.api.java.io.PrintingOutputFormat; import org.apache.flink.api.java.io.TextOutputFormat; import org.apache.flink.api.java.operators.AggregateOperator; import org.apache.flink.api.java.operators.CoGroupOperator; +import org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets; import org.apache.flink.api.java.operators.CrossOperator; +import org.apache.flink.api.java.operators.CrossOperator.DefaultCross; import org.apache.flink.api.java.operators.CustomUnaryOperation; import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.api.java.operators.DistinctOperator; import org.apache.flink.api.java.operators.FilterOperator; import org.apache.flink.api.java.operators.FlatMapOperator; -import org.apache.flink.api.java.operators.Grouping; -import org.apache.flink.api.java.operators.JoinOperator; +import org.apache.flink.api.java.operators.JoinOperator.JoinHint; +import org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.operators.MapOperator; -import org.apache.flink.api.java.operators.ProjectOperator; -import org.apache.flink.api.java.operators.ReduceGroupOperator; +import org.apache.flink.api.java.operators.ProjectOperator.Projection; +import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.ReduceOperator; -import org.apache.flink.api.java.operators.SortedGrouping; import org.apache.flink.api.java.operators.UnionOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; -import org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets; -import org.apache.flink.api.java.operators.CrossOperator.DefaultCross; -import org.apache.flink.api.java.operators.JoinOperator.JoinHint; -import org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets; -import org.apache.flink.api.java.operators.ProjectOperator.Projection; import org.apache.flink.api.java.record.functions.CrossFunction; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -137,10 +137,13 @@ public TypeInformation getType() { * @see MapOperator * @see DataSet */ - public MapOperator map(MapFunctional mapper) { + public MapOperator map(Mappable mapper) { if (mapper == null) { throw new NullPointerException("Map function must not be null."); } + if (FunctionUtils.isLambdaFunction(mapper)) { + throw new UnsupportedLambdaExpressionException(); + } return new MapOperator(this, mapper); } @@ -156,10 +159,13 @@ public MapOperator map(MapFunctional mapper) { * @see FlatMapOperator * @see DataSet */ - public FlatMapOperator flatMap(FlatMapFunction flatMapper) { + public FlatMapOperator flatMap(FlatMappable flatMapper) { if (flatMapper == null) { throw new NullPointerException("FlatMap function must not be null."); } + if (FunctionUtils.isLambdaFunction(flatMapper)) { + throw new UnsupportedLambdaExpressionException(); + } return new FlatMapOperator(this, flatMapper); } @@ -176,12 +182,13 @@ public FlatMapOperator flatMap(FlatMapFunction flatMapper) { * @see FilterOperator * @see DataSet */ - public FilterOperator filter(FilterFunctional filter) { + public FilterOperator filter(Filterable filter) { if (filter == null) { throw new NullPointerException("Filter function must not be null."); } return new FilterOperator(this, filter); } + // -------------------------------------------------------------------------------------------- // Projections @@ -278,7 +285,7 @@ public AggregateOperator min (int field) { * @see ReduceOperator * @see DataSet */ - public ReduceOperator reduce(ReduceFunctional reducer) { + public ReduceOperator reduce(Reducible reducer) { if (reducer == null) { throw new NullPointerException("Reduce function must not be null."); } @@ -295,16 +302,19 @@ public ReduceOperator reduce(ReduceFunctional reducer) { * @return A GroupReduceOperator that represents the reduced DataSet. * * @see GroupReduceFunction - * @see ReduceGroupOperator + * @see org.apache.flink.api.java.operators.GroupReduceOperator * @see DataSet */ - public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { + public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - return new ReduceGroupOperator(this, reducer); + if (FunctionUtils.isLambdaFunction(reducer)) { + throw new UnsupportedLambdaExpressionException(); + } + return new GroupReduceOperator(this, reducer); } - + // -------------------------------------------------------------------------------------------- // distinct // -------------------------------------------------------------------------------------------- @@ -375,7 +385,7 @@ public DistinctOperator distinct() { * @see SortedGrouping * @see AggregateOperator * @see ReduceOperator - * @see ReduceGroupOperator + * @see org.apache.flink.api.java.operators.GroupReduceOperator * @see DataSet */ public > UnsortedGrouping groupBy(KeySelector keyExtractor) { @@ -404,7 +414,7 @@ public > UnsortedGrouping groupBy(KeySelector k * @see SortedGrouping * @see AggregateOperator * @see ReduceOperator - * @see ReduceGroupOperator + * @see org.apache.flink.api.java.operators.GroupReduceOperator * @see DataSet */ public UnsortedGrouping groupBy(int... fields) { @@ -433,7 +443,7 @@ public UnsortedGrouping groupBy(int... fields) { * @see SortedGrouping * @see AggregateOperator * @see ReduceOperator - * @see ReduceGroupOperator + * @see org.apache.flink.api.java.operators.GroupReduceOperator * @see DataSet */ public UnsortedGrouping groupBy(String... fields) { @@ -464,7 +474,7 @@ public UnsortedGrouping groupBy(String... fields) { public JoinOperatorSets join(DataSet other) { return new JoinOperatorSets(this, other); } - + /** * Initiates a Join transformation.
* A Join transformation joins the elements of two diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java index 87cf308e10d39..0f0fe071df360 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java @@ -21,7 +21,7 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.util.Collector; /** @@ -50,7 +50,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CoGroupFunction extends AbstractRichFunction implements GenericCoGrouper { +public abstract class CoGroupFunction extends AbstractRichFunction implements CoGroupable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java new file mode 100644 index 0000000000000..b612aab010e2c --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java @@ -0,0 +1,34 @@ +/** + * 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.java.functions; + + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.util.Collector; + +import java.util.Iterator; + +public abstract class CombineFunction extends AbstractRichFunction implements FlatCombinable { + + private static final long serialVersionUID = 1L; + + @Override + public abstract void combine(Iterator values, Collector out) throws Exception; +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java index 50096a70f3aec..bc75e196596c8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java @@ -19,8 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.util.Collector; +import org.apache.flink.api.common.functions.Crossable; /** @@ -46,7 +45,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CrossFunction extends AbstractRichFunction implements GenericCrosser{ +public abstract class CrossFunction extends AbstractRichFunction implements Crossable { private static final long serialVersionUID = 1L; @@ -62,15 +61,7 @@ public abstract class CrossFunction extends AbstractRichFunction * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation * to fail and may trigger recovery. */ - public abstract OUT cross(IN1 first, IN2 second) throws Exception; - - - - /** - * This method only delegates calls to the {@link #cross(Object, Object)} method. - */ @Override - public final void cross(IN1 record1, IN2 record2, Collector out) throws Exception { - out.collect(cross(record1, record2)); - } + public abstract OUT cross(IN1 first, IN2 second) throws Exception; + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java index ff9ae452d8836..46617d8afb54f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FilterFunctional; +import org.apache.flink.api.common.functions.Filterable; /** * The abstract base class for Filter functions. A filter function take elements and evaluates a @@ -36,7 +36,7 @@ * * @param The type of the filtered elements. */ -public abstract class FilterFunction extends AbstractRichFunction implements FilterFunctional { +public abstract class FilterFunction extends AbstractRichFunction implements Filterable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java new file mode 100644 index 0000000000000..53cbcd43e30b1 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java @@ -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.java.functions; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.util.Collector; + +/** + * The abstract base class for Join functions. Join functions combine two data sets by joining their + * elements on specified keys and calling this function with each pair of joining elements. + * By default, this follows strictly the semantics of an "inner join" in SQL. + * the semantics are those of an "inner join", meaning that elements are filtered out + * if their key is not contained in the other data set. + *

+ * Per the semantics of an inner join, the function is + *

+ * The basic syntax for using Join on two data sets is as follows: + *

+ * DataSet set1 = ...; + * DataSet set2 = ...; + * + * set1.join(set2).where().equalTo().with(new MyJoinFunction()); + *
+ *

+ * {@code set1} is here considered the first input, {@code set2} the second input. + * The keys can be defined through tuple field positions or key extractors. + * See {@link org.apache.flink.api.java.operators.Keys} for details. + *

+ * The Join function is actually not a necessary part of a join operation. If no JoinFunction is provided, + * the result of the operation is a sequence of Tuple2, where the elements in the tuple are those that + * the JoinFunction would have been invoked with. + *

+ * Note: You can use a {@link CoGroupFunction} to perform an outer join. + *

+ * All functions need to be serializable, as defined in {@link java.io.Serializable}. + * + * @param The type of the elements in the first input. + * @param The type of the elements in the second input. + * @param The type of the result elements. + */ +public abstract class FlatJoinFunction extends AbstractRichFunction implements FlatJoinable { + + private static final long serialVersionUID = 1L; + + /** + * The user-defined method for performing transformations after a join. + * The method is called with matching pairs of elements from the inputs. + * + * @param first The element from first input. + * @param second The element from second input. + * @return The resulting element. + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. + */ + @Override + public abstract void join(IN1 first, IN2 second, Collector out) throws Exception; +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java index 321c6bf18991e..ab879a9573dd2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.util.Collector; /** @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class FlatMapFunction extends AbstractRichFunction implements GenericFlatMap { +public abstract class FlatMapFunction extends AbstractRichFunction implements FlatMappable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java index 1015971ddb06b..8e2f2b2c54fd0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java @@ -65,7 +65,7 @@ *

* Be aware that some annotations should only be used for functions with as single input * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs - * ({@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}). + * ({@link CrossFunction}, {@link FlatJoinFunction}, {@link CoGroupFunction}). */ public class FunctionAnnotation { @@ -103,7 +103,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function leaves certain fields of its first input values unmodified and * only "forwards" or "copies" them to the return value. The annotation is applicable to binary - * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -135,7 +135,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function leaves certain fields of its second input values unmodified and * only "forwards" or "copies" them to the return value. The annotation is applicable to binary - * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -201,7 +201,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function changes certain fields of its first input value, while leaving all * others unmodified and in place in the return value. The annotation is applicable to binary - * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -238,7 +238,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function changes certain fields of its second input value, while leaving all * others unmodified and in place in the return value. The annotation is applicable to binary - * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java index 562e8bb88f35c..c04437a289e2f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java @@ -25,8 +25,8 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.util.Collector; /** @@ -51,7 +51,7 @@ * @param Type of the elements that this function processes. * @param The type of the elements returned by the user-defined function. */ -public abstract class GroupReduceFunction extends AbstractRichFunction implements GenericGroupReduce, GenericCombine { +public abstract class GroupReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { private static final long serialVersionUID = 1L; @@ -77,7 +77,7 @@ public abstract class GroupReduceFunction extends AbstractRichFunction * This method is only ever invoked when the subclass of {@link GroupReduceFunction} * adds the {@link Combinable} annotation, or if the combinable flag is set when defining * the reduceGroup operation via - * {@link org.apache.flink.api.java.operators.ReduceGroupOperator#setCombinable(boolean)}. + * {@link org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean)}. *

* Since the reduce function will be called on the result of this method, it is important that this * method returns the same data type as it consumes. By default, this method only calls the diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java index 0624efd73f72e..14e5beff81aa8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java @@ -18,76 +18,14 @@ package org.apache.flink.api.java.functions; + import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.util.Collector; +import org.apache.flink.api.common.functions.Joinable; -/** - * The abstract base class for Join functions. Join functions combine two data sets by joining their - * elements on specified keys and calling this function with each pair of joining elements. - * By default, this follows strictly the semantics of an "inner join" in SQL. - * the semantics are those of an "inner join", meaning that elements are filtered out - * if their key is not contained in the other data set. - *

- * Per the semantics of an inner join, the function is - *

- * The basic syntax for using Join on two data sets is as follows: - *

- * DataSet set1 = ...; - * DataSet set2 = ...; - * - * set1.join(set2).where().equalTo().with(new MyJoinFunction()); - *
- *

- * {@code set1} is here considered the first input, {@code set2} the second input. - * The keys can be defined through tuple field positions or key extractors. - * See {@link org.apache.flink.api.java.operators.Keys} for details. - *

- * The Join function is actually not a necessary part of a join operation. If no JoinFunction is provided, - * the result of the operation is a sequence of Tuple2, where the elements in the tuple are those that - * the JoinFunction would have been invoked with. - *

- * Note: You can use a {@link CoGroupFunction} to perform an outer join. - *

- * All functions need to be serializable, as defined in {@link java.io.Serializable}. - * - * @param The type of the elements in the first input. - * @param The type of the elements in the second input. - * @param The type of the result elements. - */ -public abstract class JoinFunction extends AbstractRichFunction implements GenericJoiner { +public abstract class JoinFunction extends AbstractRichFunction implements Joinable { private static final long serialVersionUID = 1L; - /** - * The user-defined method for performing transformations after a join. - * The method is called with matching pairs of elements from the inputs. - * - * @param first The element from first input. - * @param second The element from second input. - * @return The resulting element. - * - * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation - * to fail and may trigger recovery. - */ - public abstract OUT join(IN1 first, IN2 second) throws Exception; - - - /** - * The user-defined method for performing transformations after a join, for operations that - * produce zero elements, or more than one element. - * By default, this method delegates to the method {@link #join(Object, Object)}. If this method - * is overridden, that method will no longer be called. - * - * @param value1 The element from first input. - * @param value2 The element from second input. - * @param out A collector to emit resulting element (zero, one, or many). - * - * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation - * to fail and may trigger recovery. - */ @Override - public void join(IN1 value1, IN2 value2, Collector out) throws Exception { - out.collect(join(value1, value2)); - } + public abstract OUT join(IN1 first, IN2 second) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java index 04a4be0c1170e..7a8d2cd044dc3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; /** * The abstract base class for Map functions. Map functions take elements and transform them, @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class MapFunction extends AbstractRichFunction implements MapFunctional { +public abstract class MapFunction extends AbstractRichFunction implements Mappable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java index 2448fb15e3cec..fd3c24935b1b7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; /** * The abstract base class for Reduce functions. Reduce functions combine groups of elements to @@ -44,7 +44,7 @@ * * @param Type of the elements that this function processes. */ -public abstract class ReduceFunction extends AbstractRichFunction implements ReduceFunctional { +public abstract class ReduceFunction extends AbstractRichFunction implements Reducible { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java index 7eb6e0037a90b..6620493f74fc8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java @@ -1,13 +1,28 @@ -package org.apache.flink.api.java.functions; - /** - * Created by kostas on 20/07/14. + * 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.java.functions; + public class UnsupportedLambdaExpressionException extends RuntimeException { private static final long serialVersionUID = -1721898801986321010L; - public UnsupportedLambdaExpressionException(String message) { - super(message); + public UnsupportedLambdaExpressionException() { + super("Java 8 lambda expressions are currently only supported in filter and reduce user-defined functions."); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index ca6ed940950c4..f0f7ca40a9131 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -24,7 +24,7 @@ import org.apache.commons.lang3.Validate; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -151,7 +151,7 @@ public AggregateOperator andMax (int field) { @SuppressWarnings("unchecked") @Override - protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase> translateToDataFlow(Operator input) { // sanity check if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) { @@ -183,8 +183,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); po.setCombinable(true); @@ -200,8 +200,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(true); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 285316982231b..09b02c3260971 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -21,8 +21,9 @@ import java.security.InvalidParameterException; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GenericCoGrouper; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -32,6 +33,7 @@ import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder; import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.operators.Keys.FieldPositionKeys; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingCoGroupOperator; @@ -53,7 +55,7 @@ */ public class CoGroupOperator extends TwoInputUdfOperator> { - private final CoGroupFunction function; + private final CoGroupable function; private final Keys keys1; private final Keys keys2; @@ -61,7 +63,7 @@ public class CoGroupOperator extends TwoInputUdfOperator input1, DataSet input2, Keys keys1, Keys keys2, - CoGroupFunction function, + CoGroupable function, TypeInformation returnType) { super(input1, input2, returnType); @@ -123,8 +125,8 @@ else if ((keys1 instanceof Keys.FieldPositionKeys int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - CoGroupOperatorBase> po = - new CoGroupOperatorBase>( + CoGroupOperatorBase> po = + new CoGroupOperatorBase>( function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); @@ -184,7 +186,7 @@ else if (keys1 instanceof Keys.SelectorFunctionKeys private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroup( Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupFunction function, + CoGroupable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -199,10 +201,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); cogroup.setFirstInput(keyMapper1); @@ -219,7 +221,7 @@ private static PlanUnwrappingCoGroupOperator tr private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupRight( int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupFunction function, + CoGroupable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -236,10 +238,10 @@ private static PlanUnwrappingCoGroupOperator tr final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -257,7 +259,7 @@ private static PlanUnwrappingCoGroupOperator tr private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupLeft( Keys.SelectorFunctionKeys rawKeys1, int[] logicalKeyPositions2, - CoGroupFunction function, + CoGroupable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -274,10 +276,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -510,10 +512,13 @@ private CoGroupOperatorWithoutFunction(Keys keys2) { * @see CoGroupFunction * @see DataSet */ - public CoGroupOperator with(CoGroupFunction function) { + public CoGroupOperator with(CoGroupable function) { if (function == null) { throw new NullPointerException("CoGroup function must not be null."); } + if (FunctionUtils.isLambdaFunction(function)) { + throw new UnsupportedLambdaExpressionException(); + } TypeInformation returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType()); return new CoGroupOperator(input1, input2, keys1, keys2, function, returnType); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 036d2928055aa..1df568e2f7cea 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -21,14 +21,15 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GenericCrosser; +import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.CrossOperatorBase; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.functions.CrossFunction; import org.apache.flink.api.java.functions.SemanticPropUtil; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.types.TypeInformation; @@ -48,10 +49,10 @@ */ public class CrossOperator extends TwoInputUdfOperator> { - private final CrossFunction function; + private final Crossable function; protected CrossOperator(DataSet input1, DataSet input2, - CrossFunction function, + Crossable function, TypeInformation returnType) { super(input1, input2, returnType); @@ -71,12 +72,12 @@ public void generateProjectionProperties(ProjectCrossFunction pcf) { } @Override - protected org.apache.flink.api.common.operators.base.CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { + protected org.apache.flink.api.common.operators.base.CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - CrossOperatorBase> po = - new CrossOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), name); + CrossOperatorBase> po = + new CrossOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), name); // set inputs po.setFirstInput(input1); po.setSecondInput(input2); @@ -108,7 +109,7 @@ public static final class DefaultCross extends CrossOperator input2; public DefaultCross(DataSet input1, DataSet input2) { - super(input1, input2, (CrossFunction>) new DefaultCrossFunction(), + super(input1, input2, (Crossable>) new DefaultCrossFunction(), new TupleTypeInfo>(input1.getType(), input2.getType())); if (input1 == null || input2 == null) { @@ -129,10 +130,13 @@ public DefaultCross(DataSet input1, DataSet input2) { * @see CrossFunction * @see DataSet */ - public CrossOperator with(CrossFunction function) { + public CrossOperator with(Crossable function) { if (function == null) { throw new NullPointerException("Cross function must not be null."); } + if (FunctionUtils.isLambdaFunction(function)) { + throw new UnsupportedLambdaExpressionException(); + } TypeInformation returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType()); return new CrossOperator(input1, input2, function, returnType); } @@ -220,7 +224,7 @@ public CrossOperator withConstantSetSecond(String... constantSetSec } } - public static final class ProjectCrossFunction extends CrossFunction { + public static final class ProjectCrossFunction implements Crossable { private static final long serialVersionUID = 1L; @@ -1398,7 +1402,7 @@ private TypeInformation[] extractFieldTypes(int[] fields, Class[] givenTyp // default join functions // -------------------------------------------------------------------------------------------- - public static final class DefaultCrossFunction extends CrossFunction> { + public static final class DefaultCrossFunction implements Crossable> { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index 034da71d95f0f..40d6c2d546bb3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -21,8 +21,9 @@ import java.util.Iterator; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; @@ -78,15 +79,17 @@ public DistinctOperator(DataSet input, Keys keys) { @Override protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase translateToDataFlow(Operator input) { - GroupReduceFunction function = new DistinctFunction(); + final GroupReduceFunction function = new DistinctFunction(); + final FlatCombinable combineFunction = new DistinctCombiner(); + String name = function.getClass().getName(); if (keys instanceof Keys.FieldPositionKeys) { int[] logicalKeyPositions = keys.computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(true); po.setInput(input); @@ -98,9 +101,10 @@ else if (keys instanceof Keys.SelectorFunctionKeys) { @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys = (Keys.SelectorFunctionKeys) keys; - + + PlanUnwrappingReduceGroupOperator po = translateSelectorFunctionDistinct( - selectorKeys, function, getInputType(), getResultType(), name, input, true); + selectorKeys, function, combineFunction, getInputType(), getResultType(), name, input, true); po.setDegreeOfParallelism(this.getParallelism()); @@ -114,7 +118,7 @@ else if (keys instanceof Keys.SelectorFunctionKeys) { // -------------------------------------------------------------------------------------------- private static PlanUnwrappingReduceGroupOperator translateSelectorFunctionDistinct( - Keys.SelectorFunctionKeys rawKeys, GroupReduceFunction function, + Keys.SelectorFunctionKeys rawKeys, GroupReduceFunction function, FlatCombinable combineFunction, TypeInformation inputType, TypeInformation outputType, String name, Operator input, boolean combinable) { @@ -124,10 +128,12 @@ private static PlanUnwrappingReduceGroupOperator transl TypeInformation> typeInfoWithKey = new TupleTypeInfo>(keys.getKeyType(), inputType); KeyExtractingMapper extractor = new KeyExtractingMapper(keys.getKeyExtractor()); + + + PlanUnwrappingReduceGroupOperator reducer = + new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - - MapOperatorBase, MapFunctional>> mapper = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, Mappable>> mapper = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); @@ -148,4 +154,16 @@ public void reduce(Iterator values, Collector out) out.collect(values.next()); } } + + public static final class DistinctCombiner implements FlatCombinable { + + private static final long serialVersionUID = 1L; + + @Override + public void combine(Iterator values, Collector out) + throws Exception { + out.collect(values.next()); + } + } + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java index 9ede5e08d017a..4733808e9e151 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.FilterFunctional; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.Filterable; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.operators.translation.PlanFilterOperator; @@ -33,10 +33,10 @@ */ public class FilterOperator extends SingleInputUdfOperator> { - protected final FilterFunctional function; + protected final Filterable function; - public FilterOperator(DataSet input, FilterFunctional function) { + public FilterOperator(DataSet input, Filterable function) { super(input, input.getType()); this.function = function; @@ -44,7 +44,7 @@ public FilterOperator(DataSet input, FilterFunctional function) { } @Override - protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java index 14c081949c9b6..d3c3aa3925951 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java @@ -18,11 +18,10 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; -import org.apache.flink.api.java.functions.FlatMapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.DataSet; @@ -36,10 +35,10 @@ */ public class FlatMapOperator extends SingleInputUdfOperator> { - protected final FlatMapFunction function; + protected final FlatMappable function; - public FlatMapOperator(DataSet input, FlatMapFunction function) { + public FlatMapOperator(DataSet input, FlatMappable function) { super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType())); this.function = function; @@ -47,11 +46,11 @@ public FlatMapOperator(DataSet input, FlatMapFunction function) { } @Override - protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); + FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java similarity index 81% rename from flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java rename to flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java index 5e993dc5d8627..1675280884cc7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java @@ -18,9 +18,9 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -28,7 +28,6 @@ import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.tuple.Tuple2; @@ -38,6 +37,7 @@ import org.apache.flink.api.java.DataSet; + /** * This operator represents the application of a "reduceGroup" function on a data set, and the * result data set produced by the function. @@ -45,26 +45,29 @@ * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. */ -public class ReduceGroupOperator extends SingleInputUdfOperator> { - - private final GroupReduceFunction function; - +public class GroupReduceOperator extends SingleInputUdfOperator> { + + private final GroupReducible function; + private final Grouping grouper; - + + // reduceFunction is a GroupReduceFunction + private boolean richFunction; + private boolean combinable; - - + /** * Constructor for a non-grouped reduce (all reduce). * * @param input The input data set to the groupReduce function. * @param function The user-defined GroupReduce function. */ - public ReduceGroupOperator(DataSet input, GroupReduceFunction function) { + public GroupReduceOperator(DataSet input, GroupReducible function) { super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType())); this.function = function; this.grouper = null; + checkCombinability(); } @@ -74,21 +77,24 @@ public ReduceGroupOperator(DataSet input, GroupReduceFunction funct * @param input The grouped input to be processed group-wise by the groupReduce function. * @param function The user-defined GroupReduce function. */ - public ReduceGroupOperator(Grouping input, GroupReduceFunction function) { + public GroupReduceOperator(Grouping input, GroupReducible function) { super(input != null ? input.getDataSet() : null, TypeExtractor.getGroupReduceReturnTypes(function, input.getDataSet().getType())); this.function = function; this.grouper = input; + checkCombinability(); - + extractSemanticAnnotationsFromUdf(function.getClass()); } - + private void checkCombinability() { - if (function instanceof GenericCombine && function.getClass().getAnnotation(Combinable.class) != null) { + if (function instanceof FlatCombinable && + function.getClass().getAnnotation(GroupReduceFunction.Combinable.class) != null) { this.combinable = true; } } + // -------------------------------------------------------------------------------------------- // Properties @@ -100,7 +106,7 @@ public boolean isCombinable() { public void setCombinable(boolean combinable) { // sanity check that the function is a subclass of the combine interface - if (combinable && !(function instanceof GenericCombine)) { + if (combinable && !(function instanceof FlatCombinable)) { throw new IllegalArgumentException("The function does not implement the combine interface."); } @@ -109,15 +115,15 @@ public void setCombinable(boolean combinable) { @Override protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase translateToDataFlow(Operator input) { - + String name = getName() != null ? getName() : function.getClass().getName(); // distinguish between grouped reduce and non-grouped reduce if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); po.setCombinable(combinable); // set input @@ -143,8 +149,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(combinable); po.setInput(input); @@ -176,7 +182,7 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { // -------------------------------------------------------------------------------------------- private static PlanUnwrappingReduceGroupOperator translateSelectorFunctionReducer( - Keys.SelectorFunctionKeys rawKeys, GroupReduceFunction function, + Keys.SelectorFunctionKeys rawKeys, GroupReducible function, TypeInformation inputType, TypeInformation outputType, String name, Operator input, boolean combinable) { @@ -189,7 +195,7 @@ private static PlanUnwrappingReduceGroupOperator transl PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - MapOperatorBase, MapFunctional>> mapper = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, Mappable>> mapper = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); @@ -199,4 +205,7 @@ private static PlanUnwrappingReduceGroupOperator transl return reducer; } + + + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index fc388fffccb82..979b8e6121071 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -22,8 +22,10 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.Joinable; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Operator; @@ -32,19 +34,22 @@ import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.FlatJoinFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.operators.Keys.FieldPositionKeys; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingJoinOperator; import org.apache.flink.api.java.operators.translation.TupleKeyExtractingMapper; +import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.types.TypeInformation; //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; +import org.apache.flink.util.Collector; //CHECKSTYLE.ON: AvoidStarImport /** @@ -147,12 +152,12 @@ protected JoinHint getJoinHint() { * @param The type of the second input DataSet of the Join transformation. * @param The type of the result of the Join transformation. * - * @see JoinFunction + * @see org.apache.flink.api.java.functions.FlatJoinFunction * @see DataSet */ public static class EquiJoin extends JoinOperator { - private final JoinFunction function; + private final FlatJoinable function; @SuppressWarnings("unused") private boolean preserve1; @@ -160,7 +165,7 @@ public static class EquiJoin extends JoinOperator { private boolean preserve2; protected EquiJoin(DataSet input1, DataSet input2, - Keys keys1, Keys keys2, JoinFunction function, + Keys keys1, Keys keys2, FlatJoinable function, TypeInformation returnType, JoinHint hint) { super(input1, input2, keys1, keys2, returnType, hint); @@ -171,14 +176,33 @@ protected EquiJoin(DataSet input1, DataSet input2, this.function = function; - if (!(function instanceof ProjectJoinFunction)) { + if (!(function instanceof ProjectFlatJoinFunction)) { extractSemanticAnnotationsFromUdf(function.getClass()); } else { - generateProjectionProperties(((ProjectJoinFunction) function)); + generateProjectionProperties(((ProjectFlatJoinFunction) function)); } } - public void generateProjectionProperties(ProjectJoinFunction pjf) { + protected EquiJoin(DataSet input1, DataSet input2, + Keys keys1, Keys keys2, FlatJoinable generatedFunction, Joinable function, + TypeInformation returnType, JoinHint hint) + { + super(input1, input2, keys1, keys2, returnType, hint); + + if (function == null) { + throw new NullPointerException(); + } + + this.function = generatedFunction; + + if (!(generatedFunction instanceof ProjectFlatJoinFunction)) { + extractSemanticAnnotationsFromUdf(function.getClass()); + } else { + generateProjectionProperties(((ProjectFlatJoinFunction) generatedFunction)); + } + } + + public void generateProjectionProperties(ProjectFlatJoinFunction pjf) { DualInputSemanticProperties props = SemanticPropUtil.createProjectionPropertiesDual(pjf.getFields(), pjf.getIsFromFirst()); setSemanticProperties(props); } @@ -238,8 +262,8 @@ else if ((super.keys1 instanceof Keys.FieldPositionKeys int[] logicalKeyPositions1 = super.keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = super.keys2.computeLogicalKeyPositions(); - JoinOperatorBase> po = - new JoinOperatorBase>(function, + JoinOperatorBase> po = + new JoinOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); @@ -298,7 +322,7 @@ else if (super.keys1 instanceof Keys.SelectorFunctionKeys private static PlanUnwrappingJoinOperator translateSelectorFunctionJoin( Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, - JoinFunction function, + FlatJoinable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -313,10 +337,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); join.setFirstInput(keyMapper1); @@ -333,7 +357,7 @@ private static PlanUnwrappingJoinOperator trans private static PlanUnwrappingJoinOperator translateSelectorFunctionJoinRight( int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, - JoinFunction function, + FlatJoinable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -350,10 +374,10 @@ private static PlanUnwrappingJoinOperator trans final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -371,7 +395,7 @@ private static PlanUnwrappingJoinOperator trans private static PlanUnwrappingJoinOperator translateSelectorFunctionJoinLeft( Keys.SelectorFunctionKeys rawKeys1, int[] logicalKeyPositions2, - JoinFunction function, + FlatJoinable function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -388,10 +412,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, MapFunctional>> keyMapper1 = - new MapOperatorBase, MapFunctional>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, MapFunctional>> keyMapper2 = - new MapOperatorBase, MapFunctional>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, Mappable>> keyMapper1 = + new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, Mappable>> keyMapper2 = + new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -424,28 +448,77 @@ protected DefaultJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { super(input1, input2, keys1, keys2, - (JoinFunction>) new DefaultJoinFunction(), + (FlatJoinFunction>) new DefaultFlatJoinFunction(), new TupleTypeInfo>(input1.getType(), input2.getType()), hint); } /** - * Finalizes a Join transformation by applying a {@link JoinFunction} to each pair of joined elements.
+ * Finalizes a Join transformation by applying a {@link org.apache.flink.api.java.functions.FlatJoinFunction} to each pair of joined elements.
* Each JoinFunction call returns exactly one element. * * @param function The JoinFunction that is called for each pair of joined elements. * @return An EquiJoin that represents the joined result DataSet * - * @see JoinFunction + * @see org.apache.flink.api.java.functions.FlatJoinFunction * @see org.apache.flink.api.java.operators.JoinOperator.EquiJoin * @see DataSet */ - public EquiJoin with(JoinFunction function) { + public EquiJoin with(FlatJoinable function) { if (function == null) { throw new NullPointerException("Join function must not be null."); } + if (FunctionUtils.isLambdaFunction(function)) { + throw new UnsupportedLambdaExpressionException(); + } TypeInformation returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type()); return new EquiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), function, returnType, getJoinHint()); } + + public EquiJoin with (Joinable function) { + if (function == null) { + throw new NullPointerException("Join function must not be null."); + } + if (FunctionUtils.isLambdaFunction(function)) { + throw new UnsupportedLambdaExpressionException(); + } + FlatJoinable generatedFunction = new GeneratedFlatJoinFunction (function); + TypeInformation returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type()); + return new EquiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint()); + } + + private static class GeneratedFlatJoinFunction extends WrappingFunction> implements FlatJoinable { + + private static final long serialVersionUID = 1L; + + private GeneratedFlatJoinFunction(Joinable wrappedFunction) { + super(wrappedFunction); + } + + @Override + public void join(IN1 left, IN2 right, Collector out) throws Exception { + out.collect (this.wrappedFunction.join(left, right)); + } + + public Joinable getWrappedFunction () { + return this.wrappedFunction; + } + } + + /* + private static class GeneratedFlatJoinFunction extends FlatJoinFunction { + + private Joinable function; + + private GeneratedFlatJoinFunction(Joinable function) { + this.function = function; + } + + @Override + public void join(IN1 first, IN2 second, Collector out) throws Exception { + out.collect(function.join(first, second)); + } + } + */ /** * Initiates a ProjectJoin transformation and projects the first join input
@@ -530,7 +603,7 @@ public static final class ProjectJoin extends EquiJoi protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo returnType) { super(input1, input2, keys1, keys2, - new ProjectJoinFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), + new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), returnType, hint); } @@ -821,20 +894,20 @@ protected DefaultJoin createJoinOperator(Keys keys2) { // default join functions // -------------------------------------------------------------------------------------------- - public static final class DefaultJoinFunction extends JoinFunction> { + public static final class DefaultFlatJoinFunction extends FlatJoinFunction> { private static final long serialVersionUID = 1L; private final Tuple2 outTuple = new Tuple2(); @Override - public Tuple2 join(T1 first, T2 second) throws Exception { + public void join(T1 first, T2 second, Collector> out) throws Exception { outTuple.f0 = first; outTuple.f1 = second; - return outTuple; + out.collect(outTuple); } } - public static final class ProjectJoinFunction extends JoinFunction { + public static final class ProjectFlatJoinFunction extends FlatJoinFunction { private static final long serialVersionUID = 1L; @@ -851,7 +924,7 @@ public static final class ProjectJoinFunction extends J * @param isFromFirst List of flags indicating whether the field should be copied from the first (true) or the second (false) input. * @param outTupleInstance An instance of an output tuple. */ - private ProjectJoinFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) { + private ProjectFlatJoinFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) { if(fields.length != isFromFirst.length) { throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); @@ -869,7 +942,7 @@ protected boolean[] getIsFromFirst() { return isFromFirst; } - public R join(T1 in1, T2 in2) { + public void join(T1 in1, T2 in2, Collector out) { for(int i=0; i= 0) { @@ -885,27 +958,33 @@ public R join(T1 in1, T2 in2) { } } } - return outTuple; + out.collect(outTuple); } } - public static final class LeftSemiJoinFunction extends JoinFunction { + public static final class LeftSemiFlatJoinFunction extends FlatJoinFunction { private static final long serialVersionUID = 1L; @Override - public T1 join(T1 left, T2 right) throws Exception { - return left; + //public T1 join(T1 left, T2 right) throws Exception { + // return left; + //} + public void join (T1 left, T2 right, Collector out) { + out.collect(left); } } - public static final class RightSemiJoinFunction extends JoinFunction { + public static final class RightSemiFlatJoinFunction extends FlatJoinFunction { private static final long serialVersionUID = 1L; @Override - public T2 join(T1 left, T2 right) throws Exception { - return right; + //public T2 join(T1 left, T2 right) throws Exception { + // return right; + //} + public void join (T1 left, T2 right, Collector out) { + out.collect(right); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java index 7e3fbcf4bdaca..ef95ab0c1235c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java @@ -18,11 +18,10 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.DataSet; @@ -34,14 +33,15 @@ * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. * - * @see MapFunction + * @see Mappable */ public class MapOperator extends SingleInputUdfOperator> { - protected final MapFunctional function; + protected final Mappable function; - public MapOperator(DataSet input, MapFunctional function) { + public MapOperator(DataSet input, Mappable function) { + super(input, TypeExtractor.getMapReturnTypes(function, input.getType())); this.function = function; @@ -49,11 +49,11 @@ public MapOperator(DataSet input, MapFunctional function) { } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); + MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index a6fed7c23c7d2..5d63b35089d14 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; @@ -51,7 +51,7 @@ public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo retur } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection " + Arrays.toString(fields); // create operator PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java index c39ab1e8e5ab8..776c8e5b2787a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java @@ -18,13 +18,12 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.MapFunctional; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.KeyRemovingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceOperator; @@ -40,11 +39,11 @@ * * @param The type of the data set reduced by the operator. * - * @see ReduceFunction + * @see Reducible */ public class ReduceOperator extends SingleInputUdfOperator> { - private final ReduceFunctional function; + private final Reducible function; private final Grouping grouper; @@ -55,7 +54,7 @@ public class ReduceOperator extends SingleInputUdfOperator input, ReduceFunctional function) { + public ReduceOperator(DataSet input, Reducible function) { super(input, input.getType()); this.function = function; @@ -65,7 +64,7 @@ public ReduceOperator(DataSet input, ReduceFunctional function) { } - public ReduceOperator(Grouping input, ReduceFunctional function) { + public ReduceOperator(Grouping input, Reducible function) { super(input.getDataSet(), input.getDataSet().getType()); this.function = function; @@ -83,8 +82,8 @@ public ReduceOperator(Grouping input, ReduceFunctional function) { if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, new int[0], name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, new int[0], name); // set input po.setInput(input); @@ -109,8 +108,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // reduce with field positions int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); // set input po.setInput(input); @@ -128,7 +127,7 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // -------------------------------------------------------------------------------------------- private static MapOperatorBase, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys rawKeys, - ReduceFunctional function, TypeInformation inputType, String name, Operator input, int dop) + Reducible function, TypeInformation inputType, String name, Operator input, int dop) { @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys = (Keys.SelectorFunctionKeys) rawKeys; @@ -139,8 +138,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || PlanUnwrappingReduceOperator reducer = new PlanUnwrappingReduceOperator(function, keys, name, inputType, typeInfoWithKey); - MapOperatorBase, MapFunctional>> keyExtractingMap = new MapOperatorBase, MapFunctional>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); - MapOperatorBase, T, MapFunctional, T>> keyRemovingMap = new MapOperatorBase, T, MapFunctional, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); + MapOperatorBase, Mappable>> keyExtractingMap = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, T, Mappable, T>> keyRemovingMap = new MapOperatorBase, T, Mappable, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); keyExtractingMap.setInput(input); reducer.setInput(keyExtractingMap); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 89c8bb222069d..9883cc9f826b0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -21,10 +21,13 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; /** * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.
@@ -73,15 +76,19 @@ protected Order[] getGroupSortOrders() { * @return A GroupReduceOperator that represents the reduced DataSet. * * @see GroupReduceFunction - * @see ReduceGroupOperator + * @see GroupReduceOperator * @see DataSet */ - public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { + public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - return new ReduceGroupOperator(this, reducer); + if (FunctionUtils.isLambdaFunction(reducer)) { + throw new UnsupportedLambdaExpressionException(); + } + return new GroupReduceOperator(this, reducer); } + // -------------------------------------------------------------------------------------------- // Group Operations diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 1d9d70d5a3516..6af6b6e6eb06c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -18,12 +18,16 @@ package org.apache.flink.api.java.operators; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; public class UnsortedGrouping extends Grouping { @@ -101,7 +105,7 @@ public AggregateOperator min (int field) { * @see ReduceOperator * @see DataSet */ - public ReduceOperator reduce(ReduceFunction reducer) { + public ReduceOperator reduce(Reducible reducer) { if (reducer == null) { throw new NullPointerException("Reduce function must not be null."); } @@ -118,16 +122,20 @@ public ReduceOperator reduce(ReduceFunction reducer) { * @return A GroupReduceOperator that represents the reduced DataSet. * * @see GroupReduceFunction - * @see ReduceGroupOperator + * @see GroupReduceOperator * @see DataSet */ - public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { + public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - return new ReduceGroupOperator(this, reducer); + if (FunctionUtils.isLambdaFunction(reducer)) { + throw new UnsupportedLambdaExpressionException(); + } + return new GroupReduceOperator(this, reducer); } + // -------------------------------------------------------------------------------------------- // Group Operations // -------------------------------------------------------------------------------------------- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java index 347fc68e95cd8..5795f37a36f49 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java @@ -18,27 +18,27 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.FilterFunctional; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.Filterable; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FilterOperatorBase; import org.apache.flink.types.TypeInformation; import org.apache.flink.util.Collector; -public class PlanFilterOperator extends FilterOperatorBase> { +public class PlanFilterOperator extends FilterOperatorBase> { - public PlanFilterOperator(FilterFunctional udf, String name, TypeInformation type) { + public PlanFilterOperator(Filterable udf, String name, TypeInformation type) { super(new FlatMapFilter(udf), new UnaryOperatorInformation(type, type), name); } - public static final class FlatMapFilter extends WrappingFunction> - implements GenericFlatMap + public static final class FlatMapFilter extends WrappingFunction> + implements FlatMappable { private static final long serialVersionUID = 1L; - private FlatMapFilter(FilterFunctional wrapped) { + private FlatMapFilter(Filterable wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java index 1ddc8d8b8e2a0..7044637de4d3f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java @@ -19,13 +19,13 @@ package org.apache.flink.api.java.operators.translation; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.types.TypeInformation; -public class PlanProjectOperator extends MapOperatorBase> { +public class PlanProjectOperator extends MapOperatorBase> { public PlanProjectOperator(int[] fields, String name, TypeInformation inType, TypeInformation outType) { super(new MapProjector(fields, outType.createSerializer().createInstance()), new UnaryOperatorInformation(inType, outType), name); @@ -33,7 +33,7 @@ public PlanProjectOperator(int[] fields, String name, TypeInformation inType, public static final class MapProjector extends AbstractRichFunction - implements MapFunctional + implements Mappable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java index 20bd3b0bff9a8..5c004a89262a9 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java @@ -20,20 +20,19 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.TypeInformation; import org.apache.flink.util.Collector; public class PlanUnwrappingCoGroupOperator - extends CoGroupOperatorBase, Tuple2, OUT, GenericCoGrouper, Tuple2, OUT>> + extends CoGroupOperatorBase, Tuple2, OUT, CoGroupable, Tuple2, OUT>> { - public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, + public PlanUnwrappingCoGroupOperator(CoGroupable udf, Keys.SelectorFunctionKeys key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -42,7 +41,7 @@ public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, + public PlanUnwrappingCoGroupOperator(CoGroupable udf, int[] key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -51,7 +50,7 @@ public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, new int[]{0}, key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, + public PlanUnwrappingCoGroupOperator(CoGroupable udf, Keys.SelectorFunctionKeys key1, int[] key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -62,15 +61,15 @@ public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, // -------------------------------------------------------------------------------------------- - public static final class TupleUnwrappingCoGrouper extends WrappingFunction> - implements GenericCoGrouper, Tuple2, OUT> + public static final class TupleUnwrappingCoGrouper extends WrappingFunction> + implements CoGroupable, Tuple2, OUT> { private static final long serialVersionUID = 1L; private final TupleUnwrappingIterator iter1; private final TupleUnwrappingIterator iter2; - private TupleUnwrappingCoGrouper(CoGroupFunction wrapped) { + private TupleUnwrappingCoGrouper(CoGroupable wrapped) { super(wrapped); this.iter1 = new TupleUnwrappingIterator(); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java index c121efe23cafd..1762fd91ceef8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java @@ -18,20 +18,19 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.JoinOperatorBase; -import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.TypeInformation; import org.apache.flink.util.Collector; public class PlanUnwrappingJoinOperator - extends JoinOperatorBase, Tuple2, OUT, GenericJoiner, Tuple2, OUT>> + extends JoinOperatorBase, Tuple2, OUT, FlatJoinable, Tuple2, OUT>> { - public PlanUnwrappingJoinOperator(JoinFunction udf, + public PlanUnwrappingJoinOperator(FlatJoinable udf, Keys.SelectorFunctionKeys key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -40,7 +39,7 @@ public PlanUnwrappingJoinOperator(JoinFunction udf, key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingJoinOperator(JoinFunction udf, + public PlanUnwrappingJoinOperator(FlatJoinable udf, int[] key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -49,7 +48,7 @@ public PlanUnwrappingJoinOperator(JoinFunction udf, new int[]{0}, key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingJoinOperator(JoinFunction udf, + public PlanUnwrappingJoinOperator(FlatJoinable udf, Keys.SelectorFunctionKeys key1, int[] key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -59,21 +58,26 @@ public PlanUnwrappingJoinOperator(JoinFunction udf, } public static final class TupleUnwrappingJoiner - extends WrappingFunction> - implements GenericJoiner, Tuple2, OUT> + extends WrappingFunction> + implements FlatJoinable, Tuple2, OUT> { private static final long serialVersionUID = 1L; - private TupleUnwrappingJoiner(JoinFunction wrapped) { + private TupleUnwrappingJoiner(FlatJoinable wrapped) { super(wrapped); } + //@SuppressWarnings("unchecked") + //@Override + //public OUT join(Tuple2 value1, Tuple2 value2) throws Exception { + // return wrappedFunction.join((I1)(value1.getField(1)), (I2)(value2.getField(1))); + //} + @SuppressWarnings("unchecked") @Override - public void join(Tuple2 value1, Tuple2 value2, - Collector out) throws Exception { - out.collect(wrappedFunction.join((I1)(value1.getField(1)), (I2)(value2.getField(1)))); + public void join (Tuple2 value1, Tuple2 value2, Collector collector) throws Exception { + wrappedFunction.join ((I1)(value1.getField(1)), (I2)(value2.getField(1)), collector); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java index 5a59664186d85..1038095aacf9c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java @@ -20,12 +20,11 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.TypeInformation; @@ -35,12 +34,12 @@ * A reduce operator that takes 2-tuples (key-value pairs), and applies the group reduce operation only * on the unwrapped values. */ -public class PlanUnwrappingReduceGroupOperator extends GroupReduceOperatorBase, OUT, GenericGroupReduce,OUT>> { +public class PlanUnwrappingReduceGroupOperator extends GroupReduceOperatorBase, OUT, GroupReducible,OUT>> { - public PlanUnwrappingReduceGroupOperator(GroupReduceFunction udf, Keys.SelectorFunctionKeys key, String name, + public PlanUnwrappingReduceGroupOperator(GroupReducible udf, Keys.SelectorFunctionKeys key, String name, TypeInformation outType, TypeInformation> typeInfoWithKey, boolean combinable) { - super(combinable ? new TupleUnwrappingCombinableGroupReducer(udf) : new TupleUnwrappingNonCombinableGroupReducer(udf), + super(combinable ? new TupleUnwrappingFlatCombinableGroupReducer((GroupReduceFunction) udf) : new TupleUnwrappingNonCombinableGroupReducer(udf), new UnaryOperatorInformation, OUT>(typeInfoWithKey, outType), key.computeLogicalKeyPositions(), name); super.setCombinable(combinable); @@ -48,9 +47,9 @@ public PlanUnwrappingReduceGroupOperator(GroupReduceFunction udf, Keys. // -------------------------------------------------------------------------------------------- - @Combinable - public static final class TupleUnwrappingCombinableGroupReducer extends WrappingFunction> - implements GenericGroupReduce, OUT>, GenericCombine> + @GroupReduceFunction.Combinable + public static final class TupleUnwrappingFlatCombinableGroupReducer extends WrappingFunction> + implements GroupReducible, OUT>, FlatCombinable> { private static final long serialVersionUID = 1L; @@ -58,7 +57,7 @@ public static final class TupleUnwrappingCombinableGroupReducer exte private TupleUnwrappingIterator iter; private TupleWrappingCollector coll; - private TupleUnwrappingCombinableGroupReducer(GroupReduceFunction wrapped) { + private TupleUnwrappingFlatCombinableGroupReducer(GroupReduceFunction wrapped) { super(wrapped); this.iter = new TupleUnwrappingIterator(); this.coll = new TupleWrappingCollector(this.iter); @@ -84,15 +83,15 @@ public String toString() { } } - public static final class TupleUnwrappingNonCombinableGroupReducer extends WrappingFunction> - implements GenericGroupReduce, OUT> + public static final class TupleUnwrappingNonCombinableGroupReducer extends WrappingFunction> + implements GroupReducible, OUT> { private static final long serialVersionUID = 1L; private final TupleUnwrappingIterator iter; - private TupleUnwrappingNonCombinableGroupReducer(GroupReduceFunction wrapped) { + private TupleUnwrappingNonCombinableGroupReducer(GroupReducible wrapped) { super(wrapped); this.iter = new TupleUnwrappingIterator(); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java index d407f675107d4..8abbae3ef069e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; import org.apache.flink.api.java.operators.Keys; @@ -30,21 +30,21 @@ * A reduce operator that takes 2-tuples (key-value pairs), and applies the reduce operation only * on the unwrapped values. */ -public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, ReduceFunctional>> { +public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, Reducible>> { - public PlanUnwrappingReduceOperator(ReduceFunctional udf, Keys.SelectorFunctionKeys key, String name, + public PlanUnwrappingReduceOperator(Reducible udf, Keys.SelectorFunctionKeys key, String name, TypeInformation type, TypeInformation> typeInfoWithKey) { super(new ReduceWrapper(udf), new UnaryOperatorInformation, Tuple2>(typeInfoWithKey, typeInfoWithKey), key.computeLogicalKeyPositions(), name); } - public static final class ReduceWrapper extends WrappingFunction> - implements ReduceFunctional> + public static final class ReduceWrapper extends WrappingFunction> + implements Reducible> { private static final long serialVersionUID = 1L; - private ReduceWrapper(ReduceFunctional wrapped) { + private ReduceWrapper(Reducible wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index 3dccd62eea057..cc4d2fde8f088 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -32,8 +32,8 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.IterationRuntimeContext; -import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Value; @@ -43,8 +43,7 @@ public abstract class WrappingFunction extends AbstractRichF private static final long serialVersionUID = 1L; protected final T wrappedFunction; - - + protected WrappingFunction(T wrappedFunction) { this.wrappedFunction = wrappedFunction; } @@ -52,12 +51,12 @@ protected WrappingFunction(T wrappedFunction) { @Override public void open(Configuration parameters) throws Exception { - openFunction(this.wrappedFunction, parameters); + FunctionUtils.openFunction(this.wrappedFunction, parameters); } @Override public void close() throws Exception { - closeFunction(this.wrappedFunction); + FunctionUtils.closeFunction(this.wrappedFunction); } @Override @@ -65,33 +64,14 @@ public void setRuntimeContext(RuntimeContext t) { super.setRuntimeContext(t); if (t instanceof IterationRuntimeContext) { - setFunctionRuntimeContext(this.wrappedFunction, new WrappingIterationRuntimeContext(t)); + FunctionUtils.setFunctionRuntimeContext(this.wrappedFunction, new WrappingIterationRuntimeContext(t)); } else{ - setFunctionRuntimeContext(this.wrappedFunction, new WrappingRuntimeContext(t)); - } - } - - private static void openFunction (Function function, Configuration parameters) throws Exception{ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.open (parameters); + FunctionUtils.setFunctionRuntimeContext(this.wrappedFunction, new WrappingRuntimeContext(t)); } } - private static void closeFunction (Function function) throws Exception{ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.close (); - } - } - private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.setRuntimeContext(context); - } - } private static class WrappingRuntimeContext implements RuntimeContext { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java index 1a3d4789e0094..c513d9971ef4d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java @@ -22,14 +22,14 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; /** * The CoGroupFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CoGroupOperator}. */ -public abstract class CoGroupFunction extends AbstractRichFunction implements GenericCoGrouper { +public abstract class CoGroupFunction extends AbstractRichFunction implements CoGroupable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java index ac5817c7c9951..093fc36499706 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java @@ -20,14 +20,13 @@ package org.apache.flink.api.java.record.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCrosser; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; /** * The CrossFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CrossOperator}. */ -public abstract class CrossFunction extends AbstractRichFunction implements GenericCrosser { +public abstract class CrossFunction extends AbstractRichFunction implements Crossable { private static final long serialVersionUID = 1L; @@ -35,14 +34,19 @@ public abstract class CrossFunction extends AbstractRichFunction implements Gene * This method must be implemented to provide a user implementation of a cross. * It is called for each element of the Cartesian product of both input sets. - * @param record1 The record from the second input. - * @param record2 The record from the second input. - * @param out A collector that collects all output records. + * @param first The record from the second input. + * @param second The record from the second input. + * @return The result of the cross UDF * * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the * runtime catches an exception, it aborts the task and lets the fail-over logic * decide whether to retry the task execution. */ + + //@Override + //public abstract void cross(Record record1, Record record2, Collector out) throws Exception; + @Override - public abstract void cross(Record record1, Record record2, Collector out) throws Exception; + public abstract Record cross(Record first, Record second) throws Exception; + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java index a37df4cd36d9c..29f9e4034d703 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java @@ -20,7 +20,7 @@ package org.apache.flink.api.java.record.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -28,7 +28,7 @@ * The JoinFunction must implementation by functions of a {@link org.apache.flink.api.java.operators.JoinOperator}. * It resembles an equality join of both inputs on their key fields. */ -public abstract class JoinFunction extends AbstractRichFunction implements GenericJoiner { +public abstract class JoinFunction extends AbstractRichFunction implements FlatJoinable { private static final long serialVersionUID = 1L; @@ -38,7 +38,7 @@ public abstract class JoinFunction extends AbstractRichFunction implements Gener * * @param value1 The record that comes from the first input. * @param value2 The record that comes from the second input. - * @param out A collector that collects all output pairs. + * @return The result of the join UDF as record * * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the * runtime catches an exception, it aborts the combine task and lets the fail-over logic diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java index 578442bef2fc5..d144936b93246 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java @@ -22,8 +22,8 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -31,7 +31,7 @@ * The ReduceFunction must be extended to provide a reducer implementation, as invoked by a * {@link org.apache.flink.api.java.operators.ReduceOperator}. */ -public abstract class ReduceFunction extends AbstractRichFunction implements GenericGroupReduce, GenericCombine { +public abstract class ReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 7ecbfdd855e1d..c80e64881168b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -31,12 +31,13 @@ import java.util.Set; import org.apache.commons.lang3.Validate; -import org.apache.flink.api.common.functions.GenericCoGrouper; -import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.api.common.functions.GenericFlatMap; -import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.Joinable; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.java.functions.InvalidTypesException; import org.apache.flink.api.java.functions.KeySelector; @@ -60,64 +61,75 @@ private TypeExtractor() { // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") - public static TypeInformation getMapReturnTypes(MapFunctional mapInterface, TypeInformation inType) { - validateInputType(MapFunctional.class, mapInterface.getClass(), 0, inType); + public static TypeInformation getMapReturnTypes(Mappable mapInterface, TypeInformation inType) { + validateInputType(Mappable.class, mapInterface.getClass(), 0, inType); if(mapInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) mapInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(MapFunctional.class, mapInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(Mappable.class, mapInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getFlatMapReturnTypes(GenericFlatMap flatMapInterface, TypeInformation inType) { - validateInputType(GenericFlatMap.class, flatMapInterface.getClass(), 0, inType); + public static TypeInformation getFlatMapReturnTypes(FlatMappable flatMapInterface, TypeInformation inType) { + validateInputType(FlatMappable.class, flatMapInterface.getClass(), 0, inType); if(flatMapInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) flatMapInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericFlatMap.class, flatMapInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(FlatMappable.class, flatMapInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getGroupReduceReturnTypes(GenericGroupReduce groupReduceInterface, + public static TypeInformation getGroupReduceReturnTypes(GroupReducible groupReduceInterface, TypeInformation inType) { - validateInputType(GenericGroupReduce.class, groupReduceInterface.getClass(), 0, inType); + validateInputType(GroupReducible.class, groupReduceInterface.getClass(), 0, inType); if(groupReduceInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) groupReduceInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericGroupReduce.class, groupReduceInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(GroupReducible.class, groupReduceInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getJoinReturnTypes(GenericJoiner joinInterface, + public static TypeInformation getJoinReturnTypes(FlatJoinable joinInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(GenericJoiner.class, joinInterface.getClass(), 0, in1Type); - validateInputType(GenericJoiner.class, joinInterface.getClass(), 1, in2Type); + validateInputType(FlatJoinable.class, joinInterface.getClass(), 0, in1Type); + validateInputType(FlatJoinable.class, joinInterface.getClass(), 1, in2Type); if(joinInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) joinInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericJoiner.class, joinInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(FlatJoinable.class, joinInterface.getClass(), 2, in1Type, in2Type); + } + + @SuppressWarnings("unchecked") + public static TypeInformation getJoinReturnTypes(Joinable joinInterface, + TypeInformation in1Type, TypeInformation in2Type) { + validateInputType(Joinable.class, joinInterface.getClass(), 0, in1Type); + validateInputType(Joinable.class, joinInterface.getClass(), 1, in2Type); + if(joinInterface instanceof ResultTypeQueryable) { + return ((ResultTypeQueryable) joinInterface).getProducedType(); + } + return new TypeExtractor().privateCreateTypeInfo(Joinable.class, joinInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") - public static TypeInformation getCoGroupReturnTypes(GenericCoGrouper coGroupInterface, + public static TypeInformation getCoGroupReturnTypes(CoGroupable coGroupInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(GenericCoGrouper.class, coGroupInterface.getClass(), 0, in1Type); - validateInputType(GenericCoGrouper.class, coGroupInterface.getClass(), 1, in2Type); + validateInputType(CoGroupable.class, coGroupInterface.getClass(), 0, in1Type); + validateInputType(CoGroupable.class, coGroupInterface.getClass(), 1, in2Type); if(coGroupInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) coGroupInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericCoGrouper.class, coGroupInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(CoGroupable.class, coGroupInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") - public static TypeInformation getCrossReturnTypes(GenericCrosser crossInterface, + public static TypeInformation getCrossReturnTypes(Crossable crossInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(GenericCrosser.class, crossInterface.getClass(), 0, in1Type); - validateInputType(GenericCrosser.class, crossInterface.getClass(), 1, in2Type); + validateInputType(Crossable.class, crossInterface.getClass(), 0, in1Type); + validateInputType(Crossable.class, crossInterface.getClass(), 1, in2Type); if(crossInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) crossInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GenericCrosser.class, crossInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(Crossable.class, crossInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java index 2a68a2a0769f7..163d8b2ea8dda 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java @@ -88,7 +88,7 @@ public T createInstance() { @Override public T copy(T from, T reuse) { for (int i = 0; i < arity; i++) { - Object copy = fieldSerializers[i].copy((T)from.getField(i), (T)reuse.getField(i)); + Object copy = fieldSerializers[i].copy(from.getField(i), reuse.getField(i)); reuse.setField(copy, i); } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java deleted file mode 100644 index 3d1300c5e3de0..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/lambdas/LambdaFunctionsTest.java +++ /dev/null @@ -1,75 +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.java.lambdas; - -import junit.framework.Assert; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.junit.Test; - -public class LambdaFunctionsTest { - private static String TEXT = "Aaaa\n" + - "Baaa\n" + - "Caaa\n" + - "Daaa\n" + - "Eaaa\n" + - "Faaa\n" + - "Gaaa\n" + - "Haaa\n" + - "Hbbb\n"; - - @Test - public void testFilterLambda () { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet text = env.fromElements(TEXT); - - try { - DataSet result = text - .filter(s -> s.startsWith("H")); - result.print(); - Assert.assertNotNull(result); - DataSet result2 = text - .filter(s -> s.startsWith("I")); - } - catch (Exception e) { - Assert.fail(); - } - - } - - @Test - public void testReduceLambda () { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet text = env.fromElements(TEXT); - - try { - DataSet result = text - .reduce ((s, t) -> s.concat(t).substring(0,t.length())); - result.print(); - - } - catch (Exception e) { - Assert.fail(); - } - - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java index db795d9edb52e..c1f6e0b05fa79 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java @@ -128,7 +128,7 @@ public void testCorrectTranslation() { assertEquals(IdentityMapper.class, worksetMapper.getUserCodeWrapper().getUserCodeClass()); assertEquals(NextWorksetMapper.class, nextWorksetMapper.getUserCodeWrapper().getUserCodeClass()); - assertEquals(SolutionWorksetJoin.class, solutionSetJoin.getUserCodeWrapper().getUserCodeClass()); + //assertEquals(SolutionWorksetJoin.class, solutionSetJoin.getUserCodeWrapper().getUserCodeClass()); assertEquals(BEFORE_NEXT_WORKSET_MAP, nextWorksetMapper.getName()); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java index 2b7705f8ce199..ccf38494baa95 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java @@ -23,14 +23,14 @@ import java.io.IOException; import java.util.Iterator; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.functions.CrossFunction; import org.apache.flink.api.java.functions.FlatMapFunction; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.InvalidTypesException; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.FlatJoinFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple; @@ -247,12 +247,12 @@ public void flatMap(Tuple3, Tuple1, Tuple2> @Test public void testSubclassOfTuple() { // use getJoinReturnTypes() - JoinFunction function = new JoinFunction() { + FlatJoinFunction function = new FlatJoinFunction() { private static final long serialVersionUID = 1L; @Override - public CustomTuple join(CustomTuple first, String second) throws Exception { - return null; + public void join(CustomTuple first, String second, Collector out) throws Exception { + out.collect(null); } }; @@ -1392,7 +1392,6 @@ public void setRuntimeContext(RuntimeContext t) { @Override public void open(Configuration parameters) throws Exception { - } @Override @@ -1417,7 +1416,7 @@ public Boolean map(String record) throws Exception { @Test public void testInterface() { - MapFunctional mapInterface = new MapFunctional() { + Mappable mapInterface = new Mappable() { @Override public Boolean map(String record) throws Exception { return null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java index fe7017161acfb..88dfa580a5804 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -30,7 +30,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public abstract class AbstractCachedBuildSideMatchDriver extends MatchDriver implements ResettablePactDriver, OT> { +public abstract class AbstractCachedBuildSideMatchDriver extends MatchDriver implements ResettablePactDriver, OT> { private volatile JoinTaskIterator matchIterator; @@ -110,7 +110,7 @@ public void prepare() throws Exception { @Override public void run() throws Exception { - final GenericJoiner matchStub = this.taskContext.getStub(); + final FlatJoinable matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); if (buildSideIndex == 0) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java index da0f222fbef9a..0765e55f0196a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java @@ -21,8 +21,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.util.MutableToRegularIteratorWrapper; @@ -37,13 +37,13 @@ * The GroupReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the GroupReduceFunction. * - * @see GenericGroupReduce + * @see org.apache.flink.api.common.functions.GroupReducible */ -public class AllGroupReduceDriver implements PactDriver, OT> { +public class AllGroupReduceDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(AllGroupReduceDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MutableObjectIterator input; @@ -54,7 +54,7 @@ public class AllGroupReduceDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericGroupReduce.class; + final Class> clazz = (Class>) (Class) GroupReducible.class; return clazz; } @@ -83,8 +83,8 @@ public void prepare() throws Exception { this.strategy = config.getDriverStrategy(); if (strategy == DriverStrategy.ALL_GROUP_COMBINE) { - if (!(this.taskContext.getStub() instanceof GenericCombine)) { - throw new Exception("Using combiner on a UDF that does not implement the combiner interface " + GenericCombine.class.getName()); + if (!(this.taskContext.getStub() instanceof FlatCombinable)) { + throw new Exception("Using combiner on a UDF that does not implement the combiner interface " + FlatCombinable.class.getName()); } } else if (strategy != DriverStrategy.ALL_GROUP_REDUCE) { @@ -105,13 +105,13 @@ public void run() throws Exception { // single UDF call with the single group if (inIter.hasNext()) { if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { - final GenericGroupReduce reducer = this.taskContext.getStub(); + final GroupReducible reducer = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); reducer.reduce(inIter, output); } else { @SuppressWarnings("unchecked") - final GenericCombine combiner = (GenericCombine) this.taskContext.getStub(); + final FlatCombinable combiner = (FlatCombinable) this.taskContext.getStub(); @SuppressWarnings("unchecked") final Collector output = (Collector) this.taskContext.getOutputCollector(); combiner.combine(inIter, output); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java index e892984605145..741a9b4f65850 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -35,13 +35,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see org.apache.flink.api.common.functions.ReduceFunctional + * @see org.apache.flink.api.common.functions.Reducible */ -public class AllReduceDriver implements PactDriver, T> { +public class AllReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(AllReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -52,7 +52,7 @@ public class AllReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -63,9 +63,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) ReduceFunctional.class; + final Class> clazz = (Class>) (Class) Reducible.class; return clazz; } @@ -94,7 +94,7 @@ public void run() throws Exception { LOG.debug(this.taskContext.formatLogString("AllReduce preprocessing done. Running Reducer code.")); } - final ReduceFunctional stub = this.taskContext.getStub(); + final Reducible stub = this.taskContext.getStub(); final MutableObjectIterator input = this.input; final TypeSerializer serializer = this.serializer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java index 3da451a3b8a80..8814c42c097e0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -41,12 +41,12 @@ * * @see org.apache.flink.api.java.record.functions.CoGroupFunction */ -public class CoGroupDriver implements PactDriver, OT> { +public class CoGroupDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(CoGroupDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CoGroupTaskIterator coGroupIterator; // the iterator that does the actual cogroup @@ -56,7 +56,7 @@ public class CoGroupDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -69,9 +69,9 @@ public int getNumberOfInputs() { @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericCoGrouper.class; + final Class> clazz = (Class>) (Class) CoGroupable.class; return clazz; } @@ -122,7 +122,7 @@ public void prepare() throws Exception @Override public void run() throws Exception { - final GenericCoGrouper coGroupStub = this.taskContext.getStub(); + final CoGroupable coGroupStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); final CoGroupTaskIterator coGroupIterator = this.coGroupIterator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java index 9d066186bf113..1bb471d2b0900 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -34,9 +34,9 @@ import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; -public class CoGroupWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { +public class CoGroupWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -53,7 +53,7 @@ public class CoGroupWithSolutionSetFirstDriver implements Resettab // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericCoGrouper.class; + final Class> clazz = (Class>) (Class) CoGroupable.class; return clazz; } @@ -123,7 +123,7 @@ public void prepare() { @Override public void run() throws Exception { - final GenericCoGrouper coGroupStub = taskContext.getStub(); + final CoGroupable coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT1 buildSideRecord = solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java index 80fa855f477ad..6afa690a80339 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -34,9 +34,9 @@ import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; -public class CoGroupWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { +public class CoGroupWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -53,7 +53,7 @@ public class CoGroupWithSolutionSetSecondDriver implements Resetta // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericCoGrouper.class; + final Class> clazz = (Class>) (Class) CoGroupable.class; return clazz; } @@ -123,7 +123,7 @@ public void prepare() { @Override public void run() throws Exception { - final GenericCoGrouper coGroupStub = taskContext.getStub(); + final CoGroupable coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT2 buildSideRecord = solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java index 7c311ed807c56..2f2bc548e9575 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericCrosser; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator; @@ -40,12 +40,12 @@ * * @see org.apache.flink.api.java.functions.CrossFunction */ -public class CrossDriver implements PactDriver, OT> { +public class CrossDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(CrossDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MemoryManager memManager; @@ -67,7 +67,7 @@ public class CrossDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -80,9 +80,9 @@ public int getNumberOfInputs() { @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericCrosser.class; + final Class> clazz = (Class>) (Class) Crossable.class; return clazz; } @@ -207,7 +207,7 @@ private void runBlockedOuterFirst() throws Exception { final T2 val2Reuse = serializer2.createInstance(); T2 val2Copy = serializer2.createInstance(); - final GenericCrosser crosser = this.taskContext.getStub(); + final Crossable crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -217,7 +217,8 @@ private void runBlockedOuterFirst() throws Exception { // for all values in the block while ((val1 = blockVals.next(val1Reuse)) != null) { val2Copy = serializer2.copy(val2, val2Copy); - crosser.cross(val1, val2Copy, collector); + collector.collect(crosser.cross(val1,val2Copy)); + //crosser.cross(val1, val2Copy, collector); } blockVals.reset(); } @@ -254,7 +255,7 @@ private void runBlockedOuterSecond() throws Exception { T2 val2; final T2 val2Reuse = serializer2.createInstance(); - final GenericCrosser crosser = this.taskContext.getStub(); + final Crossable crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -264,7 +265,8 @@ private void runBlockedOuterSecond() throws Exception { // for all values in the block while (this.running && ((val2 = blockVals.next(val2Reuse)) != null)) { val1Copy = serializer1.copy(val1, val1Copy); - crosser.cross(val1Copy, val2, collector); + collector.collect(crosser.cross(val1Copy, val2)); + //crosser.cross(val1Copy, val2, collector); } blockVals.reset(); } @@ -296,7 +298,7 @@ private void runStreamedOuterFirst() throws Exception { T2 val2; final T2 val2Reuse = serializer2.createInstance(); - final GenericCrosser crosser = this.taskContext.getStub(); + final Crossable crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -304,7 +306,8 @@ private void runStreamedOuterFirst() throws Exception { // for all values from the spilling side while (this.running && ((val2 = spillVals.next(val2Reuse)) != null)) { val1Copy = serializer1.copy(val1, val1Copy); - crosser.cross(val1Copy, val2, collector); + collector.collect(crosser.cross(val1Copy, val2)); + //crosser.cross(val1Copy, val2, collector); } spillVals.reset(); } @@ -332,7 +335,7 @@ private void runStreamedOuterSecond() throws Exception { final T2 val2Reuse = serializer2.createInstance(); T2 val2Copy = serializer2.createInstance(); - final GenericCrosser crosser = this.taskContext.getStub(); + final Crossable crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -340,7 +343,8 @@ private void runStreamedOuterSecond() throws Exception { // for all values from the spilling side while (this.running && (val1 = spillVals.next(val1Reuse)) != null) { val2Copy = serializer2.copy(val2, val2Copy); - crosser.cross(val1, val2Copy, collector); + collector.collect(crosser.cross(val1, val2Copy)); + //crosser.cross(val1, val2Copy, collector); } spillVals.reset(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java index 44f22a0504a7f..6b88c5d79a0ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -31,20 +31,20 @@ * The MapTask creates an iterator over all key-value pairs of its input and hands that to the map() method * of the MapFunction. * - * @see GenericFlatMap + * @see org.apache.flink.api.common.functions.FlatMappable * * @param The mapper's input data type. * @param The mapper's output data type. */ -public class FlatMapDriver implements PactDriver, OT> { +public class FlatMapDriver implements PactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -55,9 +55,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericFlatMap.class; + final Class> clazz = (Class>) (Class) FlatMappable.class; return clazz; } @@ -75,7 +75,7 @@ public void prepare() { public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); - final GenericFlatMap function = this.taskContext.getStub(); + final FlatMappable function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java index 1d0749c94566b..bc18b5b647c18 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.FlatCombinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.memorymanager.MemoryManager; @@ -39,12 +39,12 @@ * * @param The data type consumed and produced by the combiner. */ -public class GroupReduceCombineDriver implements PactDriver, T> { +public class GroupReduceCombineDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(GroupReduceCombineDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private CloseableInputProvider input; @@ -57,7 +57,7 @@ public class GroupReduceCombineDriver implements PactDriver // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -68,9 +68,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericCombine.class; + final Class> clazz = (Class>) (Class) FlatCombinable.class; return clazz; } @@ -111,7 +111,7 @@ public void run() throws Exception { this.serializerFactory.getSerializer(), this.comparator); // cache references on the stack - final GenericCombine stub = this.taskContext.getStub(); + final FlatCombinable stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 1ab080cc7bc36..4b0526ce42ef0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -37,13 +37,13 @@ * The GroupReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the GroupReduceFunction. * - * @see GenericGroupReduce + * @see org.apache.flink.api.common.functions.GroupReducible */ -public class GroupReduceDriver implements PactDriver, OT> { +public class GroupReduceDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(GroupReduceDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MutableObjectIterator input; @@ -56,7 +56,7 @@ public class GroupReduceDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -67,9 +67,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericGroupReduce.class; + final Class> clazz = (Class>) (Class) GroupReducible.class; return clazz; } @@ -100,7 +100,7 @@ public void run() throws Exception { final KeyGroupedIterator iter = new KeyGroupedIterator(this.input, this.serializer, this.comparator); // cache references on the stack - final GenericGroupReduce stub = this.taskContext.getStub(); + final GroupReducible stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java index b23b0cd526687..e90ca853cae47 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -32,9 +32,9 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public class JoinWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { +public class JoinWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -50,7 +50,7 @@ public class JoinWithSolutionSetFirstDriver implements ResettableP // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -61,9 +61,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericJoiner.class; + final Class> clazz = (Class>) (Class) FlatJoinable.class; return clazz; } @@ -126,7 +126,7 @@ public void prepare() { @Override public void run() throws Exception { - final GenericJoiner joinFunction = taskContext.getStub(); + final FlatJoinable joinFunction = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT1 buildSideRecord = this.solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java index 4fa5c5ab96272..97ae2d857dff2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -32,9 +32,9 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public class JoinWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { +public class JoinWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -50,7 +50,7 @@ public class JoinWithSolutionSetSecondDriver implements Resettable // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -61,9 +61,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericJoiner.class; + final Class> clazz = (Class>) (Class) FlatJoinable.class; return clazz; } @@ -126,7 +126,7 @@ public void prepare() { @Override public void run() throws Exception { - final GenericJoiner joinFunction = taskContext.getStub(); + final FlatJoinable joinFunction = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT2 buildSideRecord = this.solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java index 213faaad0eada..7c1c72e4ea05b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.MapFunctional; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -31,20 +31,20 @@ * The MapTask creates an iterator over all key-value pairs of its input and hands that to the map() method * of the MapFunction. * - * @see org.apache.flink.api.common.functions.MapFunctional + * @see org.apache.flink.api.common.functions.Mappable * * @param The mapper's input data type. * @param The mapper's output data type. */ -public class MapDriver implements PactDriver, OT> { +public class MapDriver implements PactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -55,9 +55,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) MapFunctional.class; + final Class> clazz = (Class>) (Class) Mappable.class; return clazz; } @@ -75,7 +75,7 @@ public void prepare() { public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); - final MapFunctional function = this.taskContext.getStub(); + final Mappable function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java index a29afa2c52f36..05911037e041a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -42,13 +42,13 @@ * The MatchTask matches all pairs of records that share the same key and come from different inputs. Each pair of * matching records is handed to the match() method of the JoinFunction. * - * @see GenericJoiner + * @see org.apache.flink.api.common.functions.FlatJoinable */ -public class MatchDriver implements PactDriver, OT> { +public class MatchDriver implements PactDriver, OT> { protected static final Log LOG = LogFactory.getLog(MatchDriver.class); - protected PactTaskContext, OT> taskContext; + protected PactTaskContext, OT> taskContext; private volatile JoinTaskIterator matchIterator; // the iterator that does the actual matching @@ -57,7 +57,7 @@ public class MatchDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -68,9 +68,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GenericJoiner.class; + final Class> clazz = (Class>) (Class) FlatJoinable.class; return clazz; } @@ -141,7 +141,7 @@ public void prepare() throws Exception{ @Override public void run() throws Exception { - final GenericJoiner matchStub = this.taskContext.getStub(); + final FlatJoinable matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); final JoinTaskIterator matchIterator = this.matchIterator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java index b9115ad4236b7..5017ae487afba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java @@ -24,7 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -44,7 +44,7 @@ * * @param The data type consumed and produced by the combiner. */ -public class ReduceCombineDriver implements PactDriver, T> { +public class ReduceCombineDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceCombineDriver.class); @@ -52,13 +52,13 @@ public class ReduceCombineDriver implements PactDriver, T private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32; - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private TypeSerializer serializer; private TypeComparator comparator; - private ReduceFunctional reducer; + private Reducible reducer; private Collector output; @@ -75,7 +75,7 @@ public class ReduceCombineDriver implements PactDriver, T // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -86,9 +86,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) ReduceFunctional.class; + final Class> clazz = (Class>) (Class) Reducible.class; return clazz; } @@ -168,7 +168,7 @@ private void sortAndCombine() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final ReduceFunctional function = this.reducer; + final Reducible function = this.reducer; final Collector output = this.output; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java index 88c6f383b96bf..d505bbd12ea92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -36,13 +36,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see org.apache.flink.api.common.functions.ReduceFunctional + * @see org.apache.flink.api.common.functions.Reducible */ -public class ReduceDriver implements PactDriver, T> { +public class ReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -55,7 +55,7 @@ public class ReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -66,9 +66,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) ReduceFunctional.class; + final Class> clazz = (Class>) (Class) Reducible.class; return clazz; } @@ -101,7 +101,7 @@ public void run() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final ReduceFunctional function = this.taskContext.getStub(); + final Reducible function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 07cff37372a49..cbd49143fb2e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -24,10 +24,9 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.distributions.DataDistribution; +import org.apache.flink.api.common.functions.FlatCombinable; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -499,7 +498,7 @@ protected void run() throws Exception { if (this.stub != null) { try { Configuration stubConfig = this.config.getStubParameters(); - openFunction(this.stub, stubConfig); + FunctionUtils.openFunction(this.stub, stubConfig); stubOpen = true; } catch (Throwable t) { @@ -512,7 +511,7 @@ protected void run() throws Exception { // close. We close here such that a regular close throwing an exception marks a task as failed. if (this.running && this.stub != null) { - closeFunction(this.stub); + FunctionUtils.closeFunction(this.stub); stubOpen = false; } @@ -527,9 +526,9 @@ protected void run() throws Exception { // modify accumulators.ll; if (this.stub != null) { // collect the counters from the stub - // KOSTAS - // !!! Is this.runtimeUdfContext the right thing to return here? !!! - Map> accumulators = getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); + + // !!! Is this.runtimeUdfContext the right thing to return here if this.stub.getRuntimeContext() is null? !!! + Map> accumulators = FunctionUtils.getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); RegularPactTask.reportAndClearAccumulators(getEnvironment(), accumulators, this.chainedTasks); } } @@ -537,7 +536,7 @@ protected void run() throws Exception { // close the input, but do not report any exceptions, since we already have another root cause if (stubOpen) { try { - closeFunction(this.stub); + FunctionUtils.closeFunction(this.stub); } catch (Throwable t) {} } @@ -587,7 +586,7 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - Map> chainedAccumulators = getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators(); + Map> chainedAccumulators = FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators(); AccumulatorHelper.mergeInto(accumulators, chainedAccumulators); } @@ -611,7 +610,7 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - AccumulatorHelper.resetAndClearAccumulators(getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators()); + AccumulatorHelper.resetAndClearAccumulators(FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators()); } } @@ -697,7 +696,7 @@ protected S initStub(Class stubSuperClass) throws Exception { throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + stubSuperClass.getName() + "' as is required."); } - setFunctionRuntimeContext(stub, this.runtimeUdfContext); + FunctionUtils.setFunctionRuntimeContext(stub, this.runtimeUdfContext); return stub; } catch (ClassCastException ccex) { @@ -992,13 +991,13 @@ this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalSt e.getMessage() == null ? "." : ": " + e.getMessage(), e); } - if (!(localStub instanceof GenericCombine)) { + if (!(localStub instanceof FlatCombinable)) { throw new IllegalStateException("Performing combining sort outside a reduce task!"); } @SuppressWarnings({ "rawtypes", "unchecked" }) CombiningUnilateralSortMerger cSorter = new CombiningUnilateralSortMerger( - (GenericCombine) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum], + (FlatCombinable) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum), this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum), this.config.getSpillingThresholdInput(inputNum)); @@ -1390,7 +1389,7 @@ public static void initOutputWriters(List writers) { */ public static void openUserCode(Function stub, Configuration parameters) throws Exception { try { - openFunction(stub, parameters); + FunctionUtils.openFunction(stub, parameters); } catch (Throwable t) { throw new Exception("The user defined 'open(Configuration)' method in " + stub.getClass().toString() + " caused an exception: " + t.getMessage(), t); } @@ -1407,7 +1406,7 @@ public static void openUserCode(Function stub, Configuration parameters) throws */ public static void closeUserCode(Function stub) throws Exception { try { - closeFunction(stub); + FunctionUtils.closeFunction(stub); } catch (Throwable t) { throw new Exception("The user defined 'close()' method caused an exception: " + t.getMessage(), t); } @@ -1510,34 +1509,5 @@ private static int[] asArray(List list) { return a; } - private static void openFunction (Function function, Configuration parameters) throws Exception{ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.open (parameters); - } - } - private static void closeFunction (Function function) throws Exception{ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.close (); - } - } - - private static RuntimeContext getFunctionRuntimeContext (Function function, RuntimeContext defaultContext){ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - return richFunction.getRuntimeContext(); - } - else { - return defaultContext; - } - } - - private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.setRuntimeContext(context); - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java index e46b2ad551d7d..6b848d20eba98 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.Function; /** @@ -32,7 +32,7 @@ * @param The type of stub driven by this driver. * @param The data type of the records produced by this driver. */ -public interface ResettablePactDriver extends PactDriver { +public interface ResettablePactDriver extends PactDriver { boolean isInputResettable(int inputNum); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java index dc73cb5476b16..a0e4e06b0111d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java @@ -19,25 +19,26 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.GenericFlatMap; +import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.RegularPactTask; public class ChainedFlatMapDriver extends ChainedDriver { - private GenericFlatMap mapper; + private FlatMappable mapper; // -------------------------------------------------------------------------------------------- @Override public void setup(AbstractInvokable parent) { @SuppressWarnings("unchecked") - final GenericFlatMap mapper = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GenericFlatMap.class); + final FlatMappable mapper = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatMappable.class); this.mapper = mapper; - mapper.setRuntimeContext(getUdfRuntimeContext()); + FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); } @Override @@ -54,14 +55,15 @@ public void closeTask() throws Exception { @Override public void cancelTask() { try { - this.mapper.close(); - } catch (Throwable t) { + FunctionUtils.closeFunction(this.mapper); + } + catch (Throwable t) { } } // -------------------------------------------------------------------------------------------- - public RichFunction getStub() { + public Function getStub() { return this.mapper; } @@ -84,4 +86,5 @@ public void collect(IT record) { public void close() { this.outputCollector.close(); } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java index 499a085ca47fa..a939995653abf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java @@ -20,26 +20,25 @@ package org.apache.flink.runtime.operators.chaining; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.MapFunctional; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.RegularPactTask; public class ChainedMapDriver extends ChainedDriver { - private MapFunctional mapper; + private Mappable mapper; // -------------------------------------------------------------------------------------------- @Override public void setup(AbstractInvokable parent) { @SuppressWarnings("unchecked") - final MapFunctional mapper = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, MapFunctional.class); + final Mappable mapper = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, Mappable.class); this.mapper = mapper; - setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); + FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); } @Override @@ -56,7 +55,7 @@ public void closeTask() throws Exception { @Override public void cancelTask() { try { - closeFunction(this.mapper); + FunctionUtils.closeFunction(this.mapper); } catch (Throwable t) { } } @@ -87,17 +86,4 @@ public void close() { this.outputCollector.close(); } - private static void setFunctionRuntimeContext (Function function, RuntimeContext context){ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.setRuntimeContext(context); - } - } - - private static void closeFunction (Function function) throws Exception{ - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - richFunction.close (); - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java index 63778aa59c408..6f6c72ea33e55 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java @@ -22,8 +22,9 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -51,7 +52,7 @@ public class SynchronousChainedCombineDriver extends ChainedDriver { private InMemorySorter sorter; - private GenericCombine combiner; + private FlatCombinable combiner; private TypeSerializer serializer; @@ -72,10 +73,10 @@ public void setup(AbstractInvokable parent) { this.parent = parent; @SuppressWarnings("unchecked") - final GenericCombine combiner = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GenericCombine.class); + final FlatCombinable combiner = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatCombinable.class); this.combiner = combiner; - combiner.setRuntimeContext(getUdfRuntimeContext()); + FunctionUtils.setFunctionRuntimeContext(combiner, getUdfRuntimeContext()); } @Override @@ -126,7 +127,7 @@ public void cancelTask() { // -------------------------------------------------------------------------------------------- - public RichFunction getStub() { + public Function getStub() { return this.combiner; } @@ -185,7 +186,7 @@ private void sortAndCombine() throws Exception { this.comparator); // cache references on the stack - final GenericCombine stub = this.combiner; + final FlatCombinable stub = this.combiner; final Collector output = this.outputCollector; // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java index 65af050750fe7..cf2e788cabaef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -101,7 +101,7 @@ public void close() { } @Override - public final boolean callWithNextKey(GenericJoiner matchFunction, Collector collector) + public final boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) throws Exception { if (this.hashJoin.nextRecord()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java index e12420196d6fe..e5e002ed0502b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -100,7 +100,7 @@ public void close() { } @Override - public boolean callWithNextKey(GenericJoiner matchFunction, Collector collector) + public boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) throws Exception { if (this.hashJoin.nextRecord()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 06d0ac72a807f..d87b32df8efb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -29,7 +29,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -71,7 +72,7 @@ public class CombiningUnilateralSortMerger extends UnilateralSortMerger { */ private static final Log LOG = LogFactory.getLog(CombiningUnilateralSortMerger.class); - private final GenericCombine combineStub; // the user code stub that does the combining + private final FlatCombinable combineStub; // the user code stub that does the combining private Configuration udfConfig; @@ -101,7 +102,7 @@ public class CombiningUnilateralSortMerger extends UnilateralSortMerger { * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to * perform the sort. */ - public CombiningUnilateralSortMerger(GenericCombine combineStub, MemoryManager memoryManager, IOManager ioManager, + public CombiningUnilateralSortMerger(FlatCombinable combineStub, MemoryManager memoryManager, IOManager ioManager, MutableObjectIterator input, AbstractInvokable parentTask, TypeSerializerFactory serializerFactory, TypeComparator comparator, double memoryFraction, int maxNumFileHandles, float startSpillingFraction) @@ -133,7 +134,7 @@ public CombiningUnilateralSortMerger(GenericCombine combineStub, MemoryManage * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to * perform the sort. */ - public CombiningUnilateralSortMerger(GenericCombine combineStub, MemoryManager memoryManager, IOManager ioManager, + public CombiningUnilateralSortMerger(FlatCombinable combineStub, MemoryManager memoryManager, IOManager ioManager, MutableObjectIterator input, AbstractInvokable parentTask, TypeSerializerFactory serializerFactory, TypeComparator comparator, double memoryFraction, int numSortBuffers, int maxNumFileHandles, @@ -254,12 +255,12 @@ else if (element == endMarker()) { // ------------------- Spilling Phase ------------------------ - final GenericCombine combineStub = CombiningUnilateralSortMerger.this.combineStub; + final FlatCombinable combineStub = CombiningUnilateralSortMerger.this.combineStub; // now that we are actually spilling, take the combiner, and open it try { - Configuration conf = CombiningUnilateralSortMerger.this.udfConfig; - combineStub.open(conf == null ? new Configuration() : conf); + Configuration conf = CombiningUnilateralSortMerger.this.udfConfig; + FunctionUtils.openFunction (combineStub, (conf == null ? new Configuration() : conf)); } catch (Throwable t) { throw new IOException("The user-defined combiner failed in its 'open()' method.", t); @@ -380,7 +381,7 @@ else if (element == endMarker()) { // close the user code try { - combineStub.close(); + FunctionUtils.closeFunction(combineStub); } catch (Throwable t) { throw new IOException("The user-defined combiner failed in its 'close()' method.", t); @@ -466,7 +467,7 @@ protected ChannelWithBlockCount mergeChannels(List channe this.memManager.getPageSize()); final WriterCollector collector = new WriterCollector(output, this.serializer); - final GenericCombine combineStub = CombiningUnilateralSortMerger.this.combineStub; + final FlatCombinable combineStub = CombiningUnilateralSortMerger.this.combineStub; // combine and write to disk try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java index 308e3339e2de5..74c3b3121ee4e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java @@ -25,7 +25,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -151,7 +151,7 @@ public void abort() { * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey() */ @Override - public boolean callWithNextKey(final GenericJoiner matchFunction, final Collector collector) + public boolean callWithNextKey(final FlatJoinable matchFunction, final Collector collector) throws Exception { if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { @@ -234,7 +234,7 @@ public boolean callWithNextKey(final GenericJoiner matchFunction, fin * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, - final Iterator valsN, final GenericJoiner matchFunction, final Collector collector) + final Iterator valsN, final FlatJoinable matchFunction, final Collector collector) throws Exception { this.copy1 = this.serializer1.copy(val1, this.copy1); @@ -267,7 +267,7 @@ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossSecond1withNValues(T2 val1, T1 firstValN, - Iterator valsN, GenericJoiner matchFunction, Collector collector) + Iterator valsN, FlatJoinable matchFunction, Collector collector) throws Exception { this.copy2 = this.serializer2.copy(val1, this.copy2); @@ -280,7 +280,7 @@ private void crossSecond1withNValues(T2 val1, T1 firstValN, if (valsN.hasNext()) { this.copy2 = this.serializer2.copy(val1, this.copy2); - matchFunction.join(nRec, this.copy2, collector); + matchFunction.join(nRec,this.copy2,collector); } else { matchFunction.join(nRec, val1, collector); more = false; @@ -297,7 +297,7 @@ private void crossSecond1withNValues(T2 val1, T1 firstValN, */ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, final T2 firstV2, final Iterator blockVals, - final GenericJoiner matchFunction, final Collector collector) + final FlatJoinable matchFunction, final Collector collector) throws Exception { // ================================================== @@ -411,7 +411,7 @@ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, // get instances of key and block value final T2 nextBlockVal = this.blockIt.next(); this.copy1 = this.serializer1.copy(nextSpillVal, this.copy1); - matchFunction.join(this.copy1, nextBlockVal, collector); + matchFunction.join(this.copy1, nextBlockVal, collector); } // reset block iterator diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java index 6bcb6ee3c8c5a..c2febc66c2558 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.util.Collector; @@ -60,7 +60,7 @@ public interface JoinTaskIterator * @return True, if a next key exists, false if no more keys exist. * @throws Exception Exceptions from the user code are forwarded. */ - boolean callWithNextKey(GenericJoiner matchFunction, Collector collector) throws Exception; + boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) throws Exception; /** * Aborts the matching process. This extra abort method is supplied, because a significant time may pass while diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java index e91e1007e15c3..f4ad109a88a0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; @@ -43,7 +43,7 @@ import org.junit.Assert; import org.junit.Test; -public class CachedMatchTaskTest extends DriverTestBase> +public class CachedMatchTaskTest extends DriverTestBase> { private static final long HASH_MEM = 6*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java index e8024f33adba5..e48e20f42782f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java @@ -23,12 +23,10 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; -import org.apache.flink.runtime.operators.CoGroupDriver; -import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; import org.apache.flink.types.IntValue; @@ -37,7 +35,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CoGroupTaskExternalITCase extends DriverTestBase> +public class CoGroupTaskExternalITCase extends DriverTestBase> { private static final long SORT_MEM = 3*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java index a453cf6b6432d..2d8a5e1115247 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java @@ -24,12 +24,10 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; -import org.apache.flink.runtime.operators.CoGroupDriver; -import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.CoGroupTaskExternalITCase.MockCoGroupStub; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; @@ -42,7 +40,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CoGroupTaskTest extends DriverTestBase> +public class CoGroupTaskTest extends DriverTestBase> { private static final long SORT_MEM = 3*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java index 9091809c9916c..03160752e61d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java @@ -24,10 +24,8 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.GroupReduceCombineDriver; import org.apache.flink.runtime.operators.CombineTaskTest.MockCombiningReduceStub; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; @@ -37,7 +35,7 @@ import org.junit.Test; -public class CombineTaskExternalITCase extends DriverTestBase> { +public class CombineTaskExternalITCase extends DriverTestBase> { private static final long COMBINE_MEM = 3 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java index 27fc11be2a815..b04e597b07454 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java @@ -25,12 +25,10 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.GroupReduceCombineDriver; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DriverTestBase; @@ -43,7 +41,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CombineTaskTest extends DriverTestBase> +public class CombineTaskTest extends DriverTestBase> { private static final long COMBINE_MEM = 3 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java index d663d9031780b..27e32f09ffc11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java @@ -21,9 +21,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericCrosser; -import org.apache.flink.runtime.operators.CrossDriver; -import org.apache.flink.runtime.operators.DriverStrategy; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.runtime.operators.CrossTaskTest.MockCrossStub; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; @@ -31,7 +29,7 @@ import org.junit.Test; -public class CrossTaskExternalITCase extends DriverTestBase> +public class CrossTaskExternalITCase extends DriverTestBase> { private static final long CROSS_MEM = 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java index 0e95ebb704a6b..1d5df972b2b42 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java @@ -23,10 +23,8 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericCrosser; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.api.java.record.functions.CrossFunction; -import org.apache.flink.runtime.operators.CrossDriver; -import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; @@ -36,7 +34,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CrossTaskTest extends DriverTestBase> +public class CrossTaskTest extends DriverTestBase> { private static final long CROSS_MEM = 1024 * 1024; @@ -585,10 +583,10 @@ public void run() { public static final class MockCrossStub extends CrossFunction { private static final long serialVersionUID = 1L; - + @Override - public void cross(Record record1, Record record2, Collector out) { - out.collect(record1); + public Record cross(Record record1, Record record2) throws Exception { + return record1; } } @@ -598,11 +596,11 @@ public static final class MockFailingCrossStub extends CrossFunction { private int cnt = 0; @Override - public void cross(Record record1, Record record2, Collector out) { + public Record cross(Record record1, Record record2) { if (++this.cnt >= 10) { throw new ExpectedTestException(); } - out.collect(record1); + return record1; } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java index 97dc86916bed0..c93d5e1fb93f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java @@ -21,12 +21,10 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.MatchDriver; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; import org.apache.flink.types.IntValue; @@ -35,7 +33,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class MatchTaskExternalITCase extends DriverTestBase> +public class MatchTaskExternalITCase extends DriverTestBase> { private static final long HASH_MEM = 4*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java index 6eea5464164c4..38cdaeee51ba0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java @@ -23,12 +23,10 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.MatchDriver; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; @@ -42,7 +40,7 @@ import org.junit.Assert; import org.junit.Test; -public class MatchTaskTest extends DriverTestBase> +public class MatchTaskTest extends DriverTestBase> { private static final long HASH_MEM = 6*1024*1024; @@ -951,11 +949,10 @@ public static final class MockFailingMatchStub extends JoinFunction { private int cnt = 0; @Override - public void join(Record record1, Record record2, Collector out) { + public void join(Record record1, Record record2, Collector out) throws Exception { if (++this.cnt >= 10) { throw new ExpectedTestException(); } - out.collect(record1); } } @@ -964,10 +961,11 @@ public static final class MockDelayingMatchStub extends JoinFunction { private static final long serialVersionUID = 1L; @Override - public void join(Record record1, Record record2, Collector out) { + public void join(Record record1, Record record2, Collector out) throws Exception { try { Thread.sleep(100); - } catch (InterruptedException e) { } + } catch (InterruptedException e) { + } } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java index afcbf9704f1be..90e6f4f752579 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java @@ -27,13 +27,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.GroupReduceDriver; import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; @@ -43,7 +41,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class ReduceTaskExternalITCase extends DriverTestBase> +public class ReduceTaskExternalITCase extends DriverTestBase> { private static final Log LOG = LogFactory.getLog(ReduceTaskExternalITCase.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java index 6d8ec94658c60..8d4bfd4a7a8e7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java @@ -28,13 +28,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.GroupReduceDriver; import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; @@ -48,7 +46,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class ReduceTaskTest extends DriverTestBase> +public class ReduceTaskTest extends DriverTestBase> { private static final Log LOG = LogFactory.getLog(ReduceTaskTest.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java index 8c8715e728750..4becfd8d6c158 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java @@ -23,7 +23,7 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -46,8 +46,8 @@ public class AllGroupReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -72,8 +72,8 @@ public void testAllReduceDriverImmutableEmpty() { @Test public void testAllReduceDriverImmutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -112,8 +112,8 @@ public void testAllReduceDriverImmutable() { @Test public void testAllReduceDriverMutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java index c0562b6ec6c01..ab3166df08878 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java @@ -22,7 +22,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -44,8 +44,8 @@ public class AllReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -71,8 +71,8 @@ public void testAllReduceDriverImmutableEmpty() { public void testAllReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -103,8 +103,8 @@ public void testAllReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -145,8 +145,8 @@ public void testAllReduceDriverImmutable() { public void testAllReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -176,8 +176,8 @@ public void testAllReduceDriverMutable() { Assert.assertEquals(78, res.f1.getValue()); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java index e5a01d60fe826..c5924ba380059 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java @@ -22,7 +22,7 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -46,8 +46,8 @@ public class GroupReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -74,8 +74,8 @@ public void testAllReduceDriverImmutableEmpty() { @Test public void testAllReduceDriverImmutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -110,8 +110,8 @@ public void testAllReduceDriverImmutable() { @Test public void testAllReduceDriverMutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java index 79994fd85bcdd..981e7e579d99b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java @@ -22,7 +22,7 @@ import java.util.Collections; import java.util.List; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -44,8 +44,8 @@ public class ReduceCombineDriverTest { @Test public void testImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -81,8 +81,8 @@ public void testImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -112,8 +112,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -153,8 +153,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); @@ -181,8 +181,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java index 671315baacec6..c7ba2a980d7cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java @@ -21,7 +21,7 @@ import java.util.List; -import org.apache.flink.api.common.functions.ReduceFunctional; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -43,8 +43,8 @@ public class ReduceDriverTest { @Test public void testReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -76,8 +76,8 @@ public void testReduceDriverImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -104,8 +104,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -142,8 +142,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -169,8 +169,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java index 6165a2e212f73..2d1e5b0a4f2d5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java @@ -27,7 +27,7 @@ import java.util.Map.Entry; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -412,7 +412,7 @@ public void testBuildFirstWithMixedDataTypes() { collectIntPairData(input1), collectRecordData(input2)); - final GenericJoiner matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final FlatJoinable matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); // reset the generators @@ -423,7 +423,7 @@ public void testBuildFirstWithMixedDataTypes() { // compare with iterator values BuildSecondHashMatchIterator iterator = new BuildSecondHashMatchIterator( - input1, input2, this.pairSerializer, this.pairComparator, + input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator, this.memoryManager, this.ioManager, this.parentTask, 1.0); @@ -459,7 +459,7 @@ public void testBuildSecondWithMixedDataTypes() { collectIntPairData(input1), collectRecordData(input2)); - final GenericJoiner matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final FlatJoinable matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); // reset the generators @@ -673,7 +673,7 @@ protected RecordMatchRemovingJoin(Map> map } @Override - public void join(Record rec1, Record rec2, Collector out) + public void join(Record rec1, Record rec2, Collector out) throws Exception { TestData.Key key = rec1.getField(0, TestData.Key.class); TestData.Value value1 = rec1.getField(1, TestData.Value.class); @@ -693,7 +693,7 @@ public void join(Record rec1, Record rec2, Collector out) } } - static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements GenericJoiner + static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinable { private final Map> toRemoveFrom; @@ -702,7 +702,7 @@ protected RecordIntPairMatchRemovingMatcher(Map out) + public void join(IntPair rec1, Record rec2, Collector out) throws Exception { final int k = rec1.getKey(); final int v = rec1.getValue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java index e2de4cb0e317e..93e52f6ac10fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; import org.apache.flink.runtime.operators.sort.MergeIterator; import org.apache.flink.runtime.operators.sort.MergeMatchIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; @@ -353,8 +354,7 @@ protected MatchRemovingMatcher(Map> map) { } @Override - public void join(Record rec1, Record rec2, Collector out) - { + public void join(Record rec1, Record rec2, Collector out) throws Exception { TestData.Key key = rec1.getField(0, TestData.Key.class); TestData.Value value1 = rec1.getField(1, TestData.Value.class); TestData.Value value2 = rec2.getField(1, TestData.Value.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 304d7045c5364..cb88ca73d6791 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -24,7 +24,9 @@ import junit.framework.Assert; +import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; @@ -47,7 +49,7 @@ import org.junit.After; import org.junit.BeforeClass; -public class DriverTestBase implements PactTaskContext { +public class DriverTestBase implements PactTaskContext { protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; @@ -173,7 +175,7 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { // open stub implementation try { - this.stub.open(getTaskConfig().getStubParameters()); + FunctionUtils.openFunction(this.stub, getTaskConfig().getStubParameters()); stubOpen = true; } catch (Throwable t) { @@ -185,7 +187,7 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { // close. We close here such that a regular close throwing an exception marks a task as failed. if (this.running) { - this.stub.close(); + FunctionUtils.closeFunction (this.stub); stubOpen = false; } @@ -195,7 +197,7 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { // close the input, but do not report any exceptions, since we already have another root cause if (stubOpen) { try { - this.stub.close(); + FunctionUtils.closeFunction(this.stub); } catch (Throwable t) {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index 92b6c444d94a2..8b891a90581c0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; import org.apache.flink.runtime.operators.hash.BuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.BuildSecondHashMatchIterator; import org.apache.flink.runtime.operators.sort.MergeMatchIterator; @@ -250,6 +251,7 @@ private static final class NoOpMatcher extends JoinFunction { private static final long serialVersionUID = 1L; @Override - public void join(Record rec1, Record rec2, Collector out) {} + public void join(Record rec1, Record rec2, Collector out) throws Exception { + } } } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala index 6dfd1e3b23aa2..c292100a67010 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala @@ -45,7 +45,7 @@ abstract class CrossFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JC } abstract class CrossFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CrossFunctionBase[LeftIn, RightIn, Out] with Function2[LeftIn, RightIn, Out] { - override def cross(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = { + override def cross(leftRecord: Record, rightRecord: Record) : Record = { val left = leftDeserializer.deserializeRecyclingOn(leftRecord) val right = rightDeserializer.deserializeRecyclingOn(rightRecord) val output = apply(left, right) @@ -59,31 +59,7 @@ abstract class CrossFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CrossF leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo) serializer.serialize(output, leftRecord) - out.collect(leftRecord) - } -} - -abstract class FlatCrossFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CrossFunctionBase[LeftIn, RightIn, Out] with Function2[LeftIn, RightIn, Iterator[Out]] { - override def cross(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = { - val left = leftDeserializer.deserializeRecyclingOn(leftRecord) - val right = rightDeserializer.deserializeRecyclingOn(rightRecord) - val output = apply(left, right) - - if (output.nonEmpty) { - - leftRecord.setNumFields(outputLength) - - for (field <- leftDiscard) - leftRecord.setNull(field) - - leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo) - leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo) - - for (item <- output) { - serializer.serialize(item, leftRecord) - out.collect(leftRecord) - } - } + leftRecord } } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala index 17df2c39d374b..aa04093b86fbd 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala @@ -27,7 +27,7 @@ import java.util.{ Iterator => JIterator } import org.apache.flink.api.scala._ import org.apache.flink.api.scala.analysis._ -import org.apache.flink.api.scala.functions.{CrossFunctionBase, CrossFunction, FlatCrossFunction} +import org.apache.flink.api.scala.functions.{CrossFunctionBase, CrossFunction}//, FlatCrossFunction} import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util} import org.apache.flink.api.scala.functions.DeserializingIterator import org.apache.flink.api.scala.DataSet @@ -42,7 +42,7 @@ import org.apache.flink.configuration.Configuration class CrossDataSet[LeftIn, RightIn](val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) { def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CrossMacros.map[LeftIn, RightIn, Out] - def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CrossMacros.flatMap[LeftIn, RightIn, Out] + //def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CrossMacros.flatMap[LeftIn, RightIn, Out] def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)] = macro CrossMacros.filter[LeftIn, RightIn] } @@ -65,7 +65,7 @@ object CrossMacros { implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice new CrossFunctionBase[LeftIn, RightIn, Out] { - override def cross(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = { + override def cross(leftRecord: Record, rightRecord: Record) : Record = { val left = leftDeserializer.deserializeRecyclingOn(leftRecord) val right = rightDeserializer.deserializeRecyclingOn(rightRecord) val output = fun.splice.apply(left, right) @@ -79,7 +79,7 @@ object CrossMacros { leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo) serializer.serialize(output, leftRecord) - out.collect(leftRecord) + leftRecord } } } @@ -106,67 +106,6 @@ object CrossMacros { return result } - def flatMap[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = CrossDataSet[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Iterator[Out]]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = { - import c.universe._ - - val slave = MacroContextHolder.newMacroHelper(c) - - val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn] - val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn] - val (udtOut, createUdtOut) = slave.mkUdtClass[Out] - - val stub: c.Expr[CrossFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[CrossFunction[LeftIn, RightIn, Out]]) - reify { fun.splice.asInstanceOf[CrossFunctionBase[LeftIn, RightIn, Out]] } - else reify { - implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice - implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice - implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice - new CrossFunctionBase[LeftIn, RightIn, Out] { - override def cross(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = { - val left = leftDeserializer.deserializeRecyclingOn(leftRecord) - val right = rightDeserializer.deserializeRecyclingOn(rightRecord) - val output = fun.splice.apply(left, right) - - if (output.nonEmpty) { - - leftRecord.setNumFields(outputLength) - - for (field <- leftDiscard) - leftRecord.setNull(field) - - leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo) - leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo) - - for (item <- output) { - serializer.serialize(item, leftRecord) - out.collect(leftRecord) - } - } - } - } - } - val contract = reify { - val helper: CrossDataSet[LeftIn, RightIn] = c.prefix.splice - val leftInput = helper.leftInput.contract - val rightInput = helper.rightInput.contract - val generatedStub = ClosureCleaner.clean(stub.splice) - val builder = CrossOperator.builder(generatedStub).input1(leftInput).input2(rightInput) - - val ret = new CrossOperator(builder) with TwoInputScalaOperator[LeftIn, RightIn, Out] { - override def getUDF = generatedStub.udf - override def annotations = Seq( - Annotations.getConstantFieldsFirst( - Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)), - Annotations.getConstantFieldsSecond( - Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo))) - } - new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {} - } - - val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree)) - - return result - } def filter[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag](c: Context { type PrefixType = CrossDataSet[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Boolean]): c.Expr[DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)]] = { import c.universe._ diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 80139c682a3ea..3967dfe766750 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -185,6 +185,18 @@ under the License. + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index 3d6479ba8ccc4..7f1867f66ddf3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -23,7 +23,7 @@ import java.util.Set; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; @@ -32,7 +32,7 @@ import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator; import org.apache.flink.util.Collector; -public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements GenericCoGrouper { +public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements CoGroupable { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java index 3bd049da4c23f..ad2436fb36071 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java @@ -23,7 +23,7 @@ import java.util.Set; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericJoiner; +import org.apache.flink.api.common.functions.FlatJoinable; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList; @@ -32,7 +32,7 @@ import org.apache.flink.util.Collector; public class CustomCompensatableDotProductMatch extends AbstractRichFunction implements - GenericJoiner + FlatJoinable { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java index 52f9b3cb22de6..6d9e49009db0d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java @@ -21,14 +21,14 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; import org.apache.flink.util.Collector; -public class CustomRankCombiner extends AbstractRichFunction implements GenericGroupReduce, - GenericCombine +public class CustomRankCombiner extends AbstractRichFunction implements GroupReducible, + FlatCombinable { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java index dabe7fc56a667..22c3588665f18 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.api.java.functions.CrossFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; @@ -32,6 +33,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.util.Collector; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -41,7 +43,7 @@ @RunWith(Parameterized.class) public class CrossITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 11; + private static int NUM_PROGRAMS = 12; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -392,6 +394,39 @@ public static String runProgram(int progId, String resultPath) throws Exception "4,4,Hallo Welt wieHello world\n"; } + case 12: { + /* + * check passing SAM interface instead of rich function + * funcitonality identical to test 1 + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.getSmall5TupleDataSet(env); + DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); + DataSet> crossDs = ds + .cross(ds2) + .with(new Crossable, Tuple5, Tuple2>() { + @Override + public Tuple2 cross(Tuple5 first, Tuple5 second) throws Exception { + return new Tuple2(first.f2+second.f2, first.f3+second.f3); + } + }); + + crossDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "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"; + } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java index 96174da1551fe..90a25052b4eff 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java @@ -38,7 +38,7 @@ @RunWith(Parameterized.class) public class FilterITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -333,6 +333,25 @@ public boolean filter(Tuple3 value) throws Exception { "14,5,Comment#8\n" + "15,5,Comment#9\n"; + } + case 9: { + /* + * Test passing lambda instead of function + * Functionality identical to test 3 + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> filterDs = ds. + filter(value -> value.f2.contains("world")); + filterDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n"; + } default: throw new IllegalArgumentException("Invalid program id"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java index 1c973472760a2..e3cbed434ecd3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.FlatMappable; import org.apache.flink.api.java.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -39,7 +40,7 @@ @RunWith(Parameterized.class) public class FlatMapITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 7; + private static int NUM_PROGRAMS = 8; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -372,6 +373,45 @@ public void flatMap(Tuple3 value, "55,6,Comment#14\n" + "55,6,Comment#15\n"; } + case 8: { + /* + * Pass interface instead of rich function + * Identical to test 3 + * + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> varyingTuplesMapDs = ds. + flatMap(new FlatMappable, Tuple3>() { + @Override + public void flatMap(Tuple3 value, Collector> out) throws Exception { + final int numTuples = value.f0 % 3; + for ( int i = 0; i < numTuples; i++ ) { + out.collect(value); + } + } + }); + + varyingTuplesMapDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "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"; + } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index 6556b5ed89c27..638403ddffee4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -24,6 +24,8 @@ import java.util.Iterator; import java.util.LinkedList; +import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.GroupReducible; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.KeySelector; @@ -47,7 +49,7 @@ @RunWith(Parameterized.class) public class GroupReduceITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 13; + private static int NUM_PROGRAMS = 14; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -89,326 +91,370 @@ public static Collection getConfigurations() throws FileNotFoundExcept private static class GroupReduceProgs { public static String runProgram(int progId, String resultPath) throws Exception { - - switch(progId) { - case 1: { + + switch (progId) { + case 1: { /* * check correctness of groupReduce on tuples with key field selector */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).reduceGroup(new Tuple3GroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1\n" + - "5,2\n" + - "15,3\n" + - "34,4\n" + - "65,5\n" + - "111,6\n"; - } - case 2: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).reduceGroup(new Tuple3GroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1\n" + + "5,2\n" + + "15,3\n" + + "34,4\n" + + "65,5\n" + + "111,6\n"; + } + case 2: { /* * check correctness of groupReduce on tuples with multiple key field selector */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(4,0).reduceGroup(new Tuple5GroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1,0,P-),1\n" + - "2,3,0,P-),1\n" + - "2,2,0,P-),2\n" + - "3,9,0,P-),2\n" + - "3,6,0,P-),3\n" + - "4,17,0,P-),1\n" + - "4,17,0,P-),2\n" + - "5,11,0,P-),1\n" + - "5,29,0,P-),2\n" + - "5,25,0,P-),3\n"; - } - case 3: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get5TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(4, 0).reduceGroup(new Tuple5GroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,0,P-),1\n" + + "2,3,0,P-),1\n" + + "2,2,0,P-),2\n" + + "3,9,0,P-),2\n" + + "3,6,0,P-),3\n" + + "4,17,0,P-),1\n" + + "4,17,0,P-),2\n" + + "5,11,0,P-),1\n" + + "5,29,0,P-),2\n" + + "5,25,0,P-),3\n"; + } + case 3: { /* * check correctness of groupReduce on tuples with key field selector and group sorting */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).sortGroup(2,Order.ASCENDING).reduceGroup(new Tuple3SortedGroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1,Hi\n" + - "5,2,Hello-Hello world\n" + - "15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" + - "34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" + - "65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" + - "111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n"; - - } - case 4: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(1); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup(new Tuple3SortedGroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,Hi\n" + + "5,2,Hello-Hello world\n" + + "15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" + + "34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" + + "65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" + + "111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n"; + + } + case 4: { /* * check correctness of groupReduce on tuples with key extractor */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(new KeySelector, Long>() { - private static final long serialVersionUID = 1L; - @Override - public Long getKey(Tuple3 in) { - return in.f1; - } - }).reduceGroup(new Tuple3GroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1\n" + - "5,2\n" + - "15,3\n" + - "34,4\n" + - "65,5\n" + - "111,6\n"; - - } - case 5: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(new KeySelector, Long>() { + private static final long serialVersionUID = 1L; + + @Override + public Long getKey(Tuple3 in) { + return in.f1; + } + }).reduceGroup(new Tuple3GroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1\n" + + "5,2\n" + + "15,3\n" + + "34,4\n" + + "65,5\n" + + "111,6\n"; + + } + case 5: { /* * check correctness of groupReduce on custom type with type extractor */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = ds. - groupBy(new KeySelector() { - private static final long serialVersionUID = 1L; - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - }).reduceGroup(new CustomTypeGroupReduce()); - - reduceDs.writeAsText(resultPath); - env.execute(); - - // return expected result - return "1,0,Hello!\n" + - "2,3,Hello!\n" + - "3,12,Hello!\n" + - "4,30,Hello!\n" + - "5,60,Hello!\n" + - "6,105,Hello!\n"; - } - case 6: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); + DataSet reduceDs = ds. + groupBy(new KeySelector() { + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(CustomType in) { + return in.myInt; + } + }).reduceGroup(new CustomTypeGroupReduce()); + + reduceDs.writeAsText(resultPath); + env.execute(); + + // return expected result + return "1,0,Hello!\n" + + "2,3,Hello!\n" + + "3,12,Hello!\n" + + "4,30,Hello!\n" + + "5,60,Hello!\n" + + "6,105,Hello!\n"; + } + case 6: { /* * check correctness of all-groupreduce for tuples */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds.reduceGroup(new AllAddingTuple3GroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "231,91,Hello World\n"; - } - case 7: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds.reduceGroup(new AllAddingTuple3GroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "231,91,Hello World\n"; + } + case 7: { /* * check correctness of all-groupreduce for custom types */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce()); - - reduceDs.writeAsText(resultPath); - env.execute(); - - // return expected result - return "91,210,Hello!"; - } - case 8: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); + DataSet reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce()); + + reduceDs.writeAsText(resultPath); + env.execute(); + + // return expected result + return "91,210,Hello!"; + } + case 8: { /* * check correctness of groupReduce with broadcast set */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).reduceGroup(new BCTuple3GroupReduce()).withBroadcastSet(intDs, "ints"); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1,55\n" + - "5,2,55\n" + - "15,3,55\n" + - "34,4,55\n" + - "65,5,55\n" + - "111,6,55\n"; - } - case 9: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet intDs = CollectionDataSets.getIntegerDataSet(env); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).reduceGroup(new BCTuple3GroupReduce()).withBroadcastSet(intDs, "ints"); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,55\n" + + "5,2,55\n" + + "15,3,55\n" + + "34,4,55\n" + + "65,5,55\n" + + "111,6,55\n"; + } + case 9: { /* * check correctness of groupReduce if UDF returns input objects multiple times and changes it in between */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "11,1,Hi!\n" + - "21,1,Hi again!\n" + - "12,2,Hi!\n" + - "22,2,Hi again!\n" + - "13,2,Hi!\n" + - "23,2,Hi again!\n"; - } - case 10: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "11,1,Hi!\n" + + "21,1,Hi again!\n" + + "12,2,Hi!\n" + + "22,2,Hi again!\n" + + "13,2,Hi!\n" + + "23,2,Hi again!\n"; + } + case 10: { /* * check correctness of groupReduce on custom type with key extractor and combine */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = ds. - groupBy(new KeySelector() { - private static final long serialVersionUID = 1L; - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - }).reduceGroup(new CustomTypeGroupReduceWithCombine()); - - reduceDs.writeAsText(resultPath); - env.execute(); - - // return expected result - return "1,0,test1\n" + - "2,3,test2\n" + - "3,12,test3\n" + - "4,30,test4\n" + - "5,60,test5\n" + - "6,105,test6\n"; - } - case 11: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); + DataSet reduceDs = ds. + groupBy(new KeySelector() { + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(CustomType in) { + return in.myInt; + } + }).reduceGroup(new CustomTypeGroupReduceWithCombine()); + + reduceDs.writeAsText(resultPath); + env.execute(); + + // return expected result + return "1,0,test1\n" + + "2,3,test2\n" + + "3,12,test3\n" + + "4,30,test4\n" + + "5,60,test5\n" + + "6,105,test6\n"; + } + case 11: { /* * check correctness of groupReduce on tuples with combine */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(2); // important because it determines how often the combiner is called - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,test1\n" + - "5,test2\n" + - "15,test3\n" + - "34,test4\n" + - "65,test5\n" + - "111,test6\n"; - } - // all-groupreduce with combine - case 12: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(2); // important because it determines how often the combiner is called + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,test1\n" + + "5,test2\n" + + "15,test3\n" + + "34,test4\n" + + "65,test5\n" + + "111,test6\n"; + } + // all-groupreduce with combine + case 12: { /* * check correctness of all-groupreduce for tuples with combine */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env) - .map(new IdentityMapper>()).setParallelism(4); - - Configuration cfg = new Configuration(); - cfg.setString(PactCompiler.HINT_SHIP_STRATEGY, PactCompiler.HINT_SHIP_STRATEGY_REPARTITION); - DataSet> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine()) - .withParameters(cfg); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "322,testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n"; - } - // descending sort not working - case 13: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env) + .map(new IdentityMapper>()).setParallelism(4); + + Configuration cfg = new Configuration(); + cfg.setString(PactCompiler.HINT_SHIP_STRATEGY, PactCompiler.HINT_SHIP_STRATEGY_REPARTITION); + DataSet> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine()) + .withParameters(cfg); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "322,testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n"; + } + // descending sort not working + case 13: { /* * check correctness of groupReduce on tuples with key field selector and group sorting */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).sortGroup(2,Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "1,1,Hi\n" + - "5,2,Hello world-Hello\n" + - "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + - "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + - "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + - "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - } - default: - throw new IllegalArgumentException("Invalid program id"); + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(1); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,Hi\n" + + "5,2,Hello world-Hello\n" + + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + } + case 14: { + /* + * check correctness of passing interface without combiner + * logic identical to test 9 + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds + .groupBy(1) + .reduceGroup(new GroupReducible, Tuple3>() { + @Override + public void reduce(Iterator> values, Collector> out) throws Exception { + while(values.hasNext()) { + Tuple3 t = values.next(); + + if(t.f0 < 4) { + t.f2 = "Hi!"; + t.f0 += 10; + out.collect(t); + t.f0 += 10; + t.f2 = "Hi again!"; + out.collect(t); + } + } + } + }); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "11,1,Hi!\n" + + "21,1,Hi again!\n" + + "12,2,Hi!\n" + + "22,2,Hi again!\n" + + "13,2,Hi!\n" + + "23,2,Hi again!\n"; + + } + default: { + throw new IllegalArgumentException("Invalid program id"); + } } } @@ -511,7 +557,8 @@ public void reduce(Iterator values, } } - + + public static class InputReturningTuple3GroupReduce extends GroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java index ffad949993884..e22844cdf9905 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java @@ -23,6 +23,8 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.Joinable; +import org.apache.flink.api.java.functions.FlatJoinFunction; import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; @@ -33,6 +35,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.util.Collector; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -101,7 +104,7 @@ public static String runProgram(int progId, String resultPath) throws Exception ds1.join(ds2) .where(1) .equalTo(1) - .with(new T3T5Join()); + .with(new T3T5FlatJoin()); joinDs.writeAsCsv(resultPath); env.execute(); @@ -126,7 +129,7 @@ public static String runProgram(int progId, String resultPath) throws Exception ds1.join(ds2) .where(0,1) .equalTo(0,4) - .with(new T3T5Join()); + .with(new T3T5FlatJoin()); joinDs.writeAsCsv(resultPath); env.execute(); @@ -177,7 +180,7 @@ public static String runProgram(int progId, String resultPath) throws Exception DataSet> joinDs = ds1.joinWithHuge(ds2) .where(1) .equalTo(1) - .with(new T3T5Join()); + .with(new T3T5FlatJoin()); joinDs.writeAsCsv(resultPath); env.execute(); @@ -202,7 +205,7 @@ public static String runProgram(int progId, String resultPath) throws Exception ds1.joinWithTiny(ds2) .where(1) .equalTo(1) - .with(new T3T5Join()); + .with(new T3T5FlatJoin()); joinDs.writeAsCsv(resultPath); env.execute(); @@ -292,35 +295,35 @@ public static String runProgram(int progId, String resultPath) throws Exception } case 9: { - /* - * Join on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + /* + * Join on a tuple input with key field selector and a custom type input with key extractor + */ - DataSet ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2) - .where(new KeySelector() { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - } - ) - .equalTo(0) - .with(new CustT3Join()); - - joinDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "Hi,Hi\n" + - "Hello,Hello\n" + - "Hello world,Hello\n"; - - } + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env); + DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); + DataSet> joinDs = + ds1.join(ds2) + .where(new KeySelector() { + @Override + public Integer getKey(CustomType value) { + return value.myInt; + } + } + ) + .equalTo(0) + .with(new CustT3Join()); + + joinDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "Hi,Hi\n" + + "Hello,Hello\n" + + "Hello world,Hello\n"; + + } case 10: { /* @@ -458,22 +461,23 @@ public Integer getKey(CustomType value) { } - public static class T3T5Join extends JoinFunction, Tuple5, Tuple2> { + public static class T3T5FlatJoin extends FlatJoinFunction, Tuple5, Tuple2> { @Override - public Tuple2 join(Tuple3 first, - Tuple5 second) { - - return new Tuple2(first.f2, second.f3); + public void join(Tuple3 first, + Tuple5 second, + Collector> out) { + + out.collect (new Tuple2 (first.f2, second.f3)); } - + } - public static class LeftReturningJoin extends JoinFunction, Tuple5, Tuple3> { + public static class LeftReturningJoin implements Joinable, Tuple5, Tuple3> { @Override public Tuple3 join(Tuple3 first, - Tuple5 second) { + Tuple5 second) { return first; } @@ -483,13 +487,13 @@ public static class RightReturningJoin extends JoinFunction join(Tuple3 first, - Tuple5 second) { + Tuple5 second) { return second; } } - public static class T3T5BCJoin extends JoinFunction, Tuple5, Tuple3> { + public static class T3T5BCJoin extends FlatJoinFunction, Tuple5, Tuple3> { private int broadcast; @@ -505,6 +509,7 @@ public void open(Configuration config) { } + /* @Override public Tuple3 join( Tuple3 first, @@ -512,13 +517,19 @@ public Tuple3 join( return new Tuple3(first.f2, second.f3, broadcast); } + */ + + @Override + public void join(Tuple3 first, Tuple5 second, Collector> out) throws Exception { + out.collect(new Tuple3 (first.f2, second.f3, broadcast)); + } } public static class T3CustJoin extends JoinFunction, CustomType, Tuple2> { @Override public Tuple2 join(Tuple3 first, - CustomType second) { + CustomType second) { return new Tuple2(first.f2, second.myString); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java index 83ae165698597..34504fa1e915c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java @@ -25,6 +25,7 @@ import junit.framework.Assert; +import org.apache.flink.api.common.functions.Mappable; import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -40,7 +41,7 @@ @RunWith(Parameterized.class) public class MapITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; + private static int NUM_PROGRAMS = 10; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -479,6 +480,39 @@ public Tuple3 map(Tuple3 value) { + "2,2,Hello\n" + "3,2,Hello world"; } + case 10: { + /* + * Test passing interface instead of rich function + * Identical to test 4 + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet typeConversionMapDs = ds. + map(new Mappable, String>() { + @Override + public String map(Tuple3 record) throws Exception { + return record.getField(2); + } + }); + + typeConversionMapDs.writeAsText(resultPath); + env.execute(); + + // return expected result + return "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"; + + } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java index 6cc106140f78d..534e6125b889d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java @@ -40,7 +40,7 @@ @RunWith(Parameterized.class) public class ReduceITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -270,6 +270,38 @@ public Integer getKey(CustomType in) { "65,5,Hi again!\n" + "111,6,Hi again!\n"; } + case 9: { + /* + * Pass lambda instead of rich function + * Functionality identical to test 2 + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get5TupleDataSet(env); + DataSet> reduceDs = ds + .groupBy(4,0) + .reduce((in1, in2) -> { + Tuple5 out = new Tuple5(); + out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4); + return out;}); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "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"; + + } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java index ed573be463cef..aaad08c367b22 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; import org.apache.flink.util.Collector; +import org.junit.Ignore; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -49,6 +50,7 @@ /** */ @RunWith(Parameterized.class) +//@Ignore("Test needs to be adapted to new cross signature") public class CrossITCase extends RecordAPITestBase { private static final Log LOG = LogFactory.getLog(CrossITCase.class); @@ -61,13 +63,30 @@ public CrossITCase(Configuration testConfig) { super(testConfig); } - private static final String LEFT_IN = "1 1\n2 2\n1 1\n2 2\n3 3\n4 4\n3 3\n4 4\n"; + //private static final String LEFT_IN = "1 1\n2 2\n1 1\n2 2\n3 3\n4 4\n3 3\n4 4\n"; - private static final String RIGHT_IN = "1 1\n1 2\n2 2\n2 4\n3 3\n3 6\n4 4\n4 8\n"; + //private static final String RIGHT_IN = "1 1\n1 2\n2 2\n2 4\n3 3\n3 6\n4 4\n4 8\n"; - private static final String RESULT = "4 1\n4 1\n4 2\n4 2\n5 2\n5 2\n5 4\n5 4\n6 3\n6 3\n7 4\n7 4\n" - + "5 0\n5 0\n5 1\n5 1\n6 1\n6 1\n6 3\n6 3\n7 2\n7 2\n8 3\n8 3\n" - + "6 -1\n6 -1\n6 0\n6 0\n7 0\n7 0\n8 1\n8 1\n" + "7 -2\n7 -2\n7 -1\n7 -1\n8 -1\n8 -1\n"; + //private static final String RESULT = "4 1\n4 1\n4 2\n4 2\n5 2\n5 2\n5 4\n5 4\n6 3\n6 3\n7 4\n7 4\n" + // + "5 0\n5 0\n5 1\n5 1\n6 1\n6 1\n6 3\n6 3\n7 2\n7 2\n8 3\n8 3\n" + // + "6 -1\n6 -1\n6 0\n6 0\n7 0\n7 0\n8 1\n8 1\n" + "7 -2\n7 -2\n7 -1\n7 -1\n8 -1\n8 -1\n"; + + //private static final String RESULT = "10 1\n10 1\n10 5\n10 5\n4 1\n4 1\n4 2\n4 2\n5 0\n5 0\n5 1\n," + + // "5 1\n5 2\n5 2\n5 4\n5 4\n6 -1\n6 -1\n6 0\n6 0\n6 1\n6 1\n6 3\n6 3\n6 3\n6 3\n6 6\n6 6\n7 -1\n" + + // "7 -1\n7 -2\n7 -2\n7 0\n7 0\n7 2\n7 2\n7 2\n7 2\n7 4\n7 4\n7 5\n7 5\n7 8\n7 8\n8 -1\n8 -1\n8 1\n" + + // "8 1\n8 1\n8 1\n8 3\n8 3\n8 4\n8 4\n8 7\n8 7\n9 0\n9 0\n9 2\n9 2\n9 3\n9 3\n9 6\n9 6\n"; + + //private static final String RESULT = "2 2\n4 4\n1 1\n3 3\n2 2\n4 4\n1 1\n3 3\n5 0\n5 1\n6 1\n 6 3\n" + + // "7 2\n7 5\n8 3\n8 7\n7 -2\n7 -1\n8 -1\n8 1\n9 0\n9 3\n10 1\n10 5\n4 1\n4 2\n5 2\n5 4\n6 3\n" + + // "6 6\n7 4\n7 8\n6 -1\n6 0\n7 0\n7 2\n8 1\n8 4\n9 2\n9 6\n5 0\n5 1\n6 1\n6 3\n7 2\n7 5\n 8 3\n" + + // "8 7\n7 -2\n7 -1\n8 -1\n8 1\n9 0\n9 3\n10 1\n10 5\n4 1\n4 2\n5 2\n5 4\n6 3\n6 6\n7 4\n7 8\n" + + // "6 -1\n6 0\n7 0\n7 2\n8 1\n8 4\n9 2\n9 6"; + + + private static final String LEFT_IN = "1 1\n2 2\n3 3\n"; + private static final String RIGHT_IN = "3 6\n4 4\n4 8\n"; + + private static final String RESULT = "6 6\n7 5\n7 8\n7 4\n8 3\n8 7\n8 4\n9 2\n9 6\n"; @Override protected void preSubmit() throws Exception { @@ -84,7 +103,7 @@ public static class TestCross extends CrossFunction implements Serializable { private IntValue integer = new IntValue(); @Override - public void cross(Record record1, Record record2, Collector out) { + public Record cross(Record record1, Record record2) throws Exception { string = record1.getField(1, string); int val1 = Integer.parseInt(string.toString()); string = record2.getField(1, string); @@ -95,16 +114,14 @@ public void cross(Record record1, Record record2, Collector out) { int key2 = Integer.parseInt(string.toString()); LOG.debug("Processing { [" + key1 + "," + val1 + "] , [" + key2 + "," + val2 + "] }"); - - if (val1 + val2 <= 6) { - string.setValue((key1 + key2 + 2) + ""); - integer.setValue(val2 - val1 + 1); - - record1.setField(0, string); - record1.setField(1, integer); - - out.collect(record1); - } + + string.setValue((key1 + key2 + 2) + ""); + integer.setValue(val2 - val1 + 1); + + record1.setField(0, string); + record1.setField(1, integer); + + return record1; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java index a545e051ea62b..0f58d183be1fb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java @@ -47,19 +47,19 @@ public class ComputeDistance extends CrossFunction implements Serializable { * 3: distance */ @Override - public void cross(Record dataPointRecord, Record clusterCenterRecord, Collector out) { - + public Record cross(Record dataPointRecord, Record clusterCenterRecord) throws Exception { + CoordVector dataPoint = dataPointRecord.getField(1, CoordVector.class); - + IntValue clusterCenterId = clusterCenterRecord.getField(0, IntValue.class); CoordVector clusterPoint = clusterCenterRecord.getField(1, CoordVector.class); - + this.distance.setValue(dataPoint.computeEuclidianDistance(clusterPoint)); - - // add cluster center id and distance to the data point record + + // add cluster center id and distance to the data point record dataPointRecord.setField(2, clusterCenterId); dataPointRecord.setField(3, this.distance); - - out.collect(dataPointRecord); + + return dataPointRecord; } } From b8b5c6f9900740203961bf10a9508f2f50bd4a81 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 30 Jul 2014 15:55:08 +0200 Subject: [PATCH 08/14] Propose test structure, DeltaIterationTraslationTest passing, renamed GeneratedJoinFunction --- #pom.xml# | 650 ++++++++++++++++++ flink-clients/pom.xml | 11 - flink-examples/flink-java-examples/pom.xml | 13 - flink-examples/pom.xml | 17 - flink-java/pom.xml | 16 - .../UnsupportedLambdaExpressionException.java | 6 +- .../api/java/operators/JoinOperator.java | 10 +- .../translation/WrappingFunction.java | 4 +- .../DeltaIterationTranslationTest.java | 10 +- flink-java8-tests/pom.xml | 196 ++++++ flink-java8-tests/pom.xml~ | 238 +++++++ .../lambdas/CoGroupITCase.java | 70 ++ .../javaApiOperators/lambdas/CrossITCase.java | 59 ++ .../lambdas/FilterITCase.java | 143 ++++ .../lambdas/FlatJoinITCase.java | 58 ++ .../lambdas/FlatMapITCase.java | 46 ++ .../lambdas/GroupReduceITCase.java | 84 +++ .../javaApiOperators/lambdas/JoinITCase.java | 58 ++ .../javaApiOperators/lambdas/MapITCase.java | 48 ++ .../lambdas/ReduceITCase.java | 145 ++++ .../runtime/operators/RegularPactTask.java | 2 - flink-tests/pom.xml | 14 +- .../test/javaApiOperators/FilterITCase.java | 13 +- .../test/javaApiOperators/ReduceITCase.java | 21 +- pom.xml | 3 + 25 files changed, 1839 insertions(+), 96 deletions(-) create mode 100644 #pom.xml# create mode 100644 flink-java8-tests/pom.xml create mode 100644 flink-java8-tests/pom.xml~ create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CoGroupITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CrossITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatJoinITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatMapITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/GroupReduceITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/JoinITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/MapITCase.java create mode 100644 flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java diff --git a/#pom.xml# b/#pom.xml# new file mode 100644 index 0000000000000..840b1cceac449 --- /dev/null +++ b/#pom.xml# @@ -0,0 +1,650 @@ + + + + + + org.apache + apache + 14 + + + 4.0.0 + + org.apache.flink + flink-parent + 0.6-incubating-SNAPSHOT + + flink + pom + http://flink.incubator.apache.org/ + 2014 + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/apache/incubator-flink + git@github.com:apache/incubator-flink.git + scm:git:https://git-wip-us.apache.org/repos/asf/incubator-flink.git + + + + flink-core + flink-java + flink-scala + flink-runtime + flink-compiler + flink-examples + flink-clients + flink-tests + flink-test-utils + flink-addons + flink-quickstart + flink-dist + + + + UTF-8 + UTF-8 + 1.2.1 + 2.2.0 + + + + + commons-logging + commons-logging + 1.1.1 + jar + compile + + + + log4j + log4j + 1.2.16 + jar + compile + + + + commons-codec + commons-codec + 1.3 + jar + compile + + + + com.google.guava + guava + 14.0.1 + jar + compile + + + + junit + junit + 4.7 + jar + test + + + + org.mockito + mockito-all + 1.8.5 + jar + test + + + + org.powermock + powermock-module-junit4 + 1.4.9 + jar + test + + + + org.powermock + powermock-api-mockito + 1.4.9 + jar + test + + + + org.hamcrest + hamcrest-all + 1.1 + jar + test + + + + + + + hadoop-1 + + + + !hadoop.profile + + + + ${hadoop-one.version} + + + + + + org.apache.hadoop + hadoop-core + ${hadoop.version} + + + + + + hadoop-2 + + + + hadoop.profile2 + + + + ${hadoop-two.version} + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + + + + + + + vendor-repos + + + + + cloudera-releases + https://repository.cloudera.com/artifactory/cloudera-repos + + true + + + false + + + + + + true + always + warn + + + false + never + fail + + HDPReleases + HDP Releases + http://repo.hortonworks.com/content/repositories/releases/ + default + + + + mapr-releases + http://repository.mapr.com/maven/ + false + true + + + + + + release + + + + org.apache.maven.plugins + maven-gpg-plugin + 1.4 + + + sign-artifacts + verify + + sign + + + + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.1 + + forked-path + false + ${arguments} -Psonatype-oss-release + + + + + + + + jdk8 + + false + 1.8 + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + true + + + + attach-javadocs + + jar + + + -Xdoclint:none + + + + + + + + + + + + + + + org.slf4j + slf4j-api + 1.4.3 + + + org.slf4j + slf4j-log4j12 + 1.4.3 + + + org.javassist + javassist + 3.18.1-GA + + + + + + + + + org.apache.maven.plugins + maven-surefire-report-plugin + 2.7 + + -Xmx1024m + + + + + + org.codehaus.mojo + cobertura-maven-plugin + 2.6 + + + html + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + true + true + + + + + + org.apache.rat + apache-rat-plugin + 0.10 + + + verify + + check + + + + + false + 0 + + + + AL2 + Apache License 2.0 + + + Licensed to the Apache Software Foundation (ASF) under one + + + + + + Apache License 2.0 + + + + + **/.* + **/*.prefs + + **/resources/** + + **/flink-bin/conf/slaves + + **/README.md + CHANGELOG + **/*.creole + CONTRIBUTORS + + **/*.iml + + **/target/** + + **/docs/** + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12.1 + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + /checkstyle.suppressions.file + /tools/maven/checkstyle.xml + true + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.6 + 1.6 + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + true + + + + attach-javadocs + + jar + + + + + + maven-failsafe-plugin + 2.7 + + + + integration-test + verify + + + + + -Xmx1024m + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.16 + + -Xmx1024m + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + + org.eclipse.jdt.launching.JRE_CONTAINER + + + true + true + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.3.1 + + + enforce-maven + + enforce + + + + + + [3.0.3,) + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar + + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + pl.project13.maven + + + git-commit-id-plugin + + + [2.1.5,) + + + revision + + + + + + + + + + org.apache.maven.plugins + + + maven-checkstyle-plugin + + + [2.12.1,) + + + check + + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + [1.0.0,) + + enforce + + + + + + + + + org.apache.maven.plugins + maven-remote-resources-plugin + [0.0.0,) + + process + + + + + + + + + + + + + + diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index e0ca970757306..8944385457338 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -154,17 +154,6 @@ under the License. - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 9fb6513c2aa40..ea0db5d8ede55 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -317,19 +317,6 @@ under the License. - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 34c1c7c3d4a0a..ae466811acc13 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -50,21 +50,4 @@ under the License. flink-java-examples flink-scala-examples - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - - - diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 57e58933f8537..203094391a108 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -34,22 +34,6 @@ under the License. jar - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.6 - 1.6 - - - - - - org.apache.flink diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java index 6620493f74fc8..ccc468555f53a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java @@ -18,11 +18,13 @@ package org.apache.flink.api.java.functions; -public class UnsupportedLambdaExpressionException extends RuntimeException { +import org.apache.flink.api.common.InvalidProgramException; + +public class UnsupportedLambdaExpressionException extends InvalidProgramException { private static final long serialVersionUID = -1721898801986321010L; public UnsupportedLambdaExpressionException() { - super("Java 8 lambda expressions are currently only supported in filter and reduce user-defined functions."); + super("Java 8 lambda expressions are currently supported only in filter and reduce user-defined functions."); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index 979b8e6121071..ac782a6c37c91 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -481,16 +481,16 @@ public EquiJoin with (Joinable function) { if (FunctionUtils.isLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } - FlatJoinable generatedFunction = new GeneratedFlatJoinFunction (function); + FlatJoinable generatedFunction = new WrappingFlatJoinFunction(function); TypeInformation returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type()); return new EquiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint()); } - private static class GeneratedFlatJoinFunction extends WrappingFunction> implements FlatJoinable { + private static class WrappingFlatJoinFunction extends WrappingFunction> implements FlatJoinable { private static final long serialVersionUID = 1L; - private GeneratedFlatJoinFunction(Joinable wrappedFunction) { + private WrappingFlatJoinFunction(Joinable wrappedFunction) { super(wrappedFunction); } @@ -498,10 +498,6 @@ private GeneratedFlatJoinFunction(Joinable wrappedFunction) { public void join(IN1 left, IN2 right, Collector out) throws Exception { out.collect (this.wrappedFunction.join(left, right)); } - - public Joinable getWrappedFunction () { - return this.wrappedFunction; - } } /* diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index cc4d2fde8f088..267d8796f6d3d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -71,7 +71,9 @@ public void setRuntimeContext(RuntimeContext t) { } } - + public T getWrappedFunction () { + return this.wrappedFunction; + } private static class WrappingRuntimeContext implements RuntimeContext { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java index c1f6e0b05fa79..30c8db14c0682 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java @@ -128,8 +128,14 @@ public void testCorrectTranslation() { assertEquals(IdentityMapper.class, worksetMapper.getUserCodeWrapper().getUserCodeClass()); assertEquals(NextWorksetMapper.class, nextWorksetMapper.getUserCodeWrapper().getUserCodeClass()); - //assertEquals(SolutionWorksetJoin.class, solutionSetJoin.getUserCodeWrapper().getUserCodeClass()); - + if (solutionSetJoin.getUserCodeWrapper().getUserCodeObject() instanceof WrappingFunction) { + WrappingFunction wf = (WrappingFunction) solutionSetJoin.getUserCodeWrapper().getUserCodeObject(); + assertEquals(SolutionWorksetJoin.class, wf.getWrappedFunction().getClass()); + } + else { + assertEquals(SolutionWorksetJoin.class, solutionSetJoin.getUserCodeWrapper().getUserCodeClass()); + } + assertEquals(BEFORE_NEXT_WORKSET_MAP, nextWorksetMapper.getName()); assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName()); diff --git a/flink-java8-tests/pom.xml b/flink-java8-tests/pom.xml new file mode 100644 index 0000000000000..edfd1a588fed9 --- /dev/null +++ b/flink-java8-tests/pom.xml @@ -0,0 +1,196 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-parent + 0.6-incubating-SNAPSHOT + .. + + + flink-java8-tests + flink-java8-tests + + jar + + + + org.apache.flink + flink-core + ${project.version} + + + org.apache.flink + flink-compiler + ${project.version} + + + org.apache.flink + flink-runtime + ${project.version} + + + org.apache.flink + flink-clients + ${project.version} + + + org.apache.flink + flink-java + ${project.version} + + + junit + junit + 4.7 + + + org.apache.flink + flink-scala + ${project.version} + test + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + org.apache.flink + flink-java-examples + ${project.version} + test + + + org.apache.flink + flink-scala-examples + ${project.version} + test + + + org.apache.flink + flink-tests + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + WARN + + once + -Xmx1024m + + + + maven-failsafe-plugin + + + WARN + + always + 1 + false + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + org.apache.maven.plugins + + + maven-assembly-plugin + + + [2.4,) + + + single + + + + + + + + + + + + + + diff --git a/flink-java8-tests/pom.xml~ b/flink-java8-tests/pom.xml~ new file mode 100644 index 0000000000000..5ddb54c24c836 --- /dev/null +++ b/flink-java8-tests/pom.xml~ @@ -0,0 +1,238 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-tests + 0.6-incubating-SNAPSHOT + .. + + + flink-java8-tests + flink-java8-tests + + jar + + + + org.apache.flink + flink-core + ${project.version} + + + org.apache.flink + flink-compiler + ${project.version} + + + org.apache.flink + flink-runtime + ${project.version} + + + org.apache.flink + flink-clients + ${project.version} + + + org.apache.flink + flink-java + ${project.version} + + + junit + junit + 4.7 + + + org.apache.flink + flink-scala + ${project.version} + test + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + org.apache.flink + flink-java-examples + ${project.version} + test + + + org.apache.flink + flink-scala-examples + ${project.version} + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + WARN + + + **/*TestBase*.class + + once + -Xmx1024m + + + + maven-failsafe-plugin + + + WARN + + always + 1 + false + + + + + maven-assembly-plugin + 2.4 + + + create-test-dependency + process-test-classes + + single + + + + + org.apache.flink.test.util.testjar.KMeansForTest + + + maven + false + + src/test/assembly/test-assembly.xml + + + + + + + + + maven-clean-plugin + 2.5 + + + remove-kmeansfortest + process-test-classes + + clean + + + true + + + ${project.build.testOutputDirectory} + + **/testjar/*.class + + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + org.apache.maven.plugins + + + maven-assembly-plugin + + + [2.4,) + + + single + + + + + + + + + + + + + + diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CoGroupITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CoGroupITCase.java new file mode 100644 index 0000000000000..c4172494175b0 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CoGroupITCase.java @@ -0,0 +1,70 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class CoGroupITCase implements Serializable { + + @Test + public void testCoGroupLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> left = env.fromElements( + new Tuple2(1, "hello"), + new Tuple2(2, "what's"), + new Tuple2(2, "up") + ); + DataSet> right = env.fromElements( + new Tuple2(1, "not"), + new Tuple2(1, "much"), + new Tuple2(2, "really") + ); + DataSet> joined = left.coGroup(right).where(0).equalTo(0) + .with((values1, values2, out) -> { + int sum = 0; + String conc = ""; + while (values1.hasNext()) { + sum += values1.next().f0; + conc += values1.next().f1; + } + while (values2.hasNext()) { + sum += values2.next().f0; + conc += values2.next().f1; + } + }); + env.execute(); + + + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CrossITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CrossITCase.java new file mode 100644 index 0000000000000..f8d217e856903 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/CrossITCase.java @@ -0,0 +1,59 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class CrossITCase implements Serializable { + + @Test + public void testCrossLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> left = env.fromElements( + new Tuple2(1, "hello"), + new Tuple2(2, "what's"), + new Tuple2(2, "up") + ); + DataSet> right = env.fromElements( + new Tuple2(1, "not"), + new Tuple2(1, "much"), + new Tuple2(2, "really") + ); + DataSet> joined = left.cross(right) + .with((t,s) -> new Tuple2 (t.f0 + s.f0, t.f1 + " " + s.f1)); + + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} + + diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java new file mode 100644 index 0000000000000..b6c13dcb84714 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java @@ -0,0 +1,143 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.FilterFunction; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +@RunWith(Parameterized.class) +public class FilterITCase extends JavaProgramTestBase { + + + public static DataSet> get3TupleDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(); + data.add(new Tuple3(1,1l,"Hi")); + data.add(new Tuple3(2,2l,"Hello")); + data.add(new Tuple3(3,2l,"Hello world")); + data.add(new Tuple3(4,3l,"Hello world, how are you?")); + data.add(new Tuple3(5,3l,"I am fine.")); + data.add(new Tuple3(6,3l,"Luke Skywalker")); + data.add(new Tuple3(7,4l,"Comment#1")); + data.add(new Tuple3(8,4l,"Comment#2")); + data.add(new Tuple3(9,4l,"Comment#3")); + data.add(new Tuple3(10,4l,"Comment#4")); + data.add(new Tuple3(11,5l,"Comment#5")); + data.add(new Tuple3(12,5l,"Comment#6")); + data.add(new Tuple3(13,5l,"Comment#7")); + data.add(new Tuple3(14,5l,"Comment#8")); + data.add(new Tuple3(15,5l,"Comment#9")); + data.add(new Tuple3(16,6l,"Comment#10")); + data.add(new Tuple3(17,6l,"Comment#11")); + data.add(new Tuple3(18,6l,"Comment#12")); + data.add(new Tuple3(19,6l,"Comment#13")); + data.add(new Tuple3(20,6l,"Comment#14")); + data.add(new Tuple3(21,6l,"Comment#15")); + + Collections.shuffle(data); + + return env.fromCollection(data); + } + + private static int NUM_PROGRAMS = 1; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public FilterITCase(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = FilterProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class FilterProgs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test lambda filter + * Functionality identical to org.apache.flink.test.javaApiOperators.FilterITCase test 3 + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = get3TupleDataSet(env); + DataSet> filterDs = ds. + filter(value -> value.f2.contains("world")); + filterDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + + } + + } + +} + diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatJoinITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatJoinITCase.java new file mode 100644 index 0000000000000..043b4e81d4fa5 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatJoinITCase.java @@ -0,0 +1,58 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class FlatJoinITCase implements Serializable { + + @Test + public void testFlatJoinLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> left = env.fromElements( + new Tuple2(1, "hello"), + new Tuple2(2, "what's"), + new Tuple2(2, "up") + ); + DataSet> right = env.fromElements( + new Tuple2(1, "not"), + new Tuple2(1, "much"), + new Tuple2(2, "really") + ); + DataSet> joined = left.join(right).where(0).equalTo(0) + .with((t,s,out) -> out.collect(new Tuple2(t.f0, t.f1 + " " + s.f1))); + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} + + diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatMapITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatMapITCase.java new file mode 100644 index 0000000000000..55f507ca0a45d --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FlatMapITCase.java @@ -0,0 +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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class FlatMapITCase implements Serializable { + + @Test + public void testFlatMapLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet stringDs = env.fromElements("aa", "ab", "ac", "ad"); + DataSet flatMappedDs = stringDs.flatMap((s, out) -> out.collect(s.replace("a", "b"))); + env.execute(); + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/GroupReduceITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/GroupReduceITCase.java new file mode 100644 index 0000000000000..494aff64c601e --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/GroupReduceITCase.java @@ -0,0 +1,84 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class GroupReduceITCase implements Serializable { + + @Test + public void testAllGroupReduceLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet stringDs = env.fromElements("aa", "ab", "ac", "ad"); + DataSet concatDs = stringDs.reduceGroup((values, out) -> { + String conc = ""; + while (values.hasNext()) { + String s = values.next(); + conc = conc.concat(s); + } + out.collect(conc); + }); + env.execute(); + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } + + @Test + public void testGroupReduceLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> stringDs = env.fromElements( + new Tuple2(1, "aa"), + new Tuple2(2, "ab"), + new Tuple2(1, "ac"), + new Tuple2(2, "ad") + ); + DataSet concatDs = stringDs + .groupBy(0) + .reduceGroup((values, out) -> { + String conc = ""; + while (values.hasNext()) { + String s = values.next().f1; + conc = conc.concat(s); + } + out.collect(conc); + }); + env.execute(); + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} \ No newline at end of file diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/JoinITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/JoinITCase.java new file mode 100644 index 0000000000000..3f4f6965aec51 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/JoinITCase.java @@ -0,0 +1,58 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class JoinITCase implements Serializable { + + @Test + public void testJoinLambda() { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> left = env.fromElements( + new Tuple2(1, "hello"), + new Tuple2(2, "what's"), + new Tuple2(2, "up") + ); + DataSet> right = env.fromElements( + new Tuple2(1, "not"), + new Tuple2(1, "much"), + new Tuple2(2, "really") + ); + DataSet> joined = left.join(right).where(0).equalTo(0) + .with((t,s) -> new Tuple2(t.f0, t.f1 + " " + t.f1)); + + } catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } catch (Exception e) { + Assert.fail(); + } + } +} + diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/MapITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/MapITCase.java new file mode 100644 index 0000000000000..3af360b6414b2 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/MapITCase.java @@ -0,0 +1,48 @@ +/** + * 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.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; + +public class MapITCase implements Serializable{ + + @Test + public void TestMapLambda () { + try { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet stringDs = env.fromElements("aa", "ab", "ac", "ad"); + DataSet mappedDs = stringDs.map (s -> s.replace("a", "b")); + env.execute(); + } + catch (UnsupportedLambdaExpressionException e) { + // Success + return; + } + catch (Exception e) { + Assert.fail(); + } + } +} diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java new file mode 100644 index 0000000000000..b63f26690e098 --- /dev/null +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java @@ -0,0 +1,145 @@ +package org.apache.flink.test.javaApiOperators.lambdas; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.typeutils.BasicTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +@RunWith(Parameterized.class) +public class ReduceITCase extends JavaProgramTestBase { + + public static DataSet> get5TupleDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(); + data.add(new Tuple5(1,1l,0,"Hallo",1l)); + data.add(new Tuple5(2,2l,1,"Hallo Welt",2l)); + data.add(new Tuple5(2,3l,2,"Hallo Welt wie",1l)); + data.add(new Tuple5(3,4l,3,"Hallo Welt wie gehts?",2l)); + data.add(new Tuple5(3,5l,4,"ABC",2l)); + data.add(new Tuple5(3,6l,5,"BCD",3l)); + data.add(new Tuple5(4,7l,6,"CDE",2l)); + data.add(new Tuple5(4,8l,7,"DEF",1l)); + data.add(new Tuple5(4,9l,8,"EFG",1l)); + data.add(new Tuple5(4,10l,9,"FGH",2l)); + data.add(new Tuple5(5,11l,10,"GHI",1l)); + data.add(new Tuple5(5,12l,11,"HIJ",3l)); + data.add(new Tuple5(5,13l,12,"IJK",3l)); + data.add(new Tuple5(5,14l,13,"JKL",2l)); + data.add(new Tuple5(5,15l,14,"KLM",2l)); + + Collections.shuffle(data); + + TupleTypeInfo> type = new + TupleTypeInfo>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO + ); + + return env.fromCollection(data, type); + } + + private static int NUM_PROGRAMS = 1; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public ReduceITCase(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = ReduceProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class ReduceProgs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test reduce with lambda + * Functionality identical to org.apache.flink.test.javaApiOperators.ReduceITCase test 2 + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = get5TupleDataSet(env); + DataSet> reduceDs = ds + .groupBy(4, 0) + .reduce((in1, in2) -> { + Tuple5 out = new Tuple5(); + out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4); + return out; + }); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "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"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + + } + + } + + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index cbd49143fb2e4..204b60b866cf2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -526,8 +526,6 @@ protected void run() throws Exception { // modify accumulators.ll; if (this.stub != null) { // collect the counters from the stub - - // !!! Is this.runtimeUdfContext the right thing to return here if this.stub.getRuntimeContext() is null? !!! Map> accumulators = FunctionUtils.getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); RegularPactTask.reportAndClearAccumulators(getEnvironment(), accumulators, this.chainedTasks); } diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 3967dfe766750..467ce38e89e1f 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -35,6 +35,7 @@ under the License. jar + org.apache.flink @@ -184,19 +185,6 @@ under the License. - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java index 90a25052b4eff..40c7ab42c2781 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.Filterable; import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -336,22 +337,26 @@ public boolean filter(Tuple3 value) throws Exception { } case 9: { /* - * Test passing lambda instead of function - * Functionality identical to test 3 + * Passing interface instead of function + * Functionality identical to org.apache.flink.test.javaApiOperators.FilterITCase test 3 */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. - filter(value -> value.f2.contains("world")); + filter(new Filterable>() { + @Override + public boolean filter(Tuple3 value) throws Exception { + return value.f2.contains("world"); + } + }); filterDs.writeAsCsv(resultPath); env.execute(); // return expected result return "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"; - } default: throw new IllegalArgumentException("Invalid program id"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java index 534e6125b889d..76040961ead57 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.Reducible; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple3; @@ -272,19 +273,23 @@ public Integer getKey(CustomType in) { } case 9: { /* - * Pass lambda instead of rich function - * Functionality identical to test 2 + * Passing interface instead of function + * Functionality identical to org.apache.flink.test.javaApiOperators.ReduceITCase test 2 */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); DataSet> reduceDs = ds - .groupBy(4,0) - .reduce((in1, in2) -> { - Tuple5 out = new Tuple5(); - out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4); - return out;}); + .groupBy(4, 0) + .reduce(new Reducible>() { + @Override + public Tuple5 reduce(Tuple5 in1, Tuple5 in2) throws Exception { + Tuple5 out = new Tuple5(); + out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4); + return out; + } + }); reduceDs.writeAsCsv(resultPath); env.execute(); @@ -302,7 +307,7 @@ public Integer getKey(CustomType in) { "5,25,0,P-),3\n"; } - default: + default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/pom.xml b/pom.xml index 840b1cceac449..083f78999f5ae 100644 --- a/pom.xml +++ b/pom.xml @@ -282,6 +282,9 @@ under the License. false 1.8 + + flink-java8-tests + From 5c9c149f6b2bcb76e6ec17e877dbd9f58c509a85 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 30 Jul 2014 15:57:55 +0200 Subject: [PATCH 09/14] delete accidentally added file --- #pom.xml# | 650 ------------------------------------------------------ 1 file changed, 650 deletions(-) delete mode 100644 #pom.xml# diff --git a/#pom.xml# b/#pom.xml# deleted file mode 100644 index 840b1cceac449..0000000000000 --- a/#pom.xml# +++ /dev/null @@ -1,650 +0,0 @@ - - - - - - org.apache - apache - 14 - - - 4.0.0 - - org.apache.flink - flink-parent - 0.6-incubating-SNAPSHOT - - flink - pom - http://flink.incubator.apache.org/ - 2014 - - - - The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - https://github.com/apache/incubator-flink - git@github.com:apache/incubator-flink.git - scm:git:https://git-wip-us.apache.org/repos/asf/incubator-flink.git - - - - flink-core - flink-java - flink-scala - flink-runtime - flink-compiler - flink-examples - flink-clients - flink-tests - flink-test-utils - flink-addons - flink-quickstart - flink-dist - - - - UTF-8 - UTF-8 - 1.2.1 - 2.2.0 - - - - - commons-logging - commons-logging - 1.1.1 - jar - compile - - - - log4j - log4j - 1.2.16 - jar - compile - - - - commons-codec - commons-codec - 1.3 - jar - compile - - - - com.google.guava - guava - 14.0.1 - jar - compile - - - - junit - junit - 4.7 - jar - test - - - - org.mockito - mockito-all - 1.8.5 - jar - test - - - - org.powermock - powermock-module-junit4 - 1.4.9 - jar - test - - - - org.powermock - powermock-api-mockito - 1.4.9 - jar - test - - - - org.hamcrest - hamcrest-all - 1.1 - jar - test - - - - - - - hadoop-1 - - - - !hadoop.profile - - - - ${hadoop-one.version} - - - - - - org.apache.hadoop - hadoop-core - ${hadoop.version} - - - - - - hadoop-2 - - - - hadoop.profile2 - - - - ${hadoop-two.version} - - - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - - - - - - - vendor-repos - - - - - cloudera-releases - https://repository.cloudera.com/artifactory/cloudera-repos - - true - - - false - - - - - - true - always - warn - - - false - never - fail - - HDPReleases - HDP Releases - http://repo.hortonworks.com/content/repositories/releases/ - default - - - - mapr-releases - http://repository.mapr.com/maven/ - false - true - - - - - - release - - - - org.apache.maven.plugins - maven-gpg-plugin - 1.4 - - - sign-artifacts - verify - - sign - - - - - - - - - org.apache.maven.plugins - maven-release-plugin - 2.1 - - forked-path - false - ${arguments} -Psonatype-oss-release - - - - - - - - jdk8 - - false - 1.8 - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.9.1 - - true - - - - attach-javadocs - - jar - - - -Xdoclint:none - - - - - - - - - - - - - - - org.slf4j - slf4j-api - 1.4.3 - - - org.slf4j - slf4j-log4j12 - 1.4.3 - - - org.javassist - javassist - 3.18.1-GA - - - - - - - - - org.apache.maven.plugins - maven-surefire-report-plugin - 2.7 - - -Xmx1024m - - - - - - org.codehaus.mojo - cobertura-maven-plugin - 2.6 - - - html - - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.4 - - - - true - true - - - - - - org.apache.rat - apache-rat-plugin - 0.10 - - - verify - - check - - - - - false - 0 - - - - AL2 - Apache License 2.0 - - - Licensed to the Apache Software Foundation (ASF) under one - - - - - - Apache License 2.0 - - - - - **/.* - **/*.prefs - - **/resources/** - - **/flink-bin/conf/slaves - - **/README.md - CHANGELOG - **/*.creole - CONTRIBUTORS - - **/*.iml - - **/target/** - - **/docs/** - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.12.1 - - - validate - validate - - check - - - - - /tools/maven/suppressions.xml - /checkstyle.suppressions.file - /tools/maven/checkstyle.xml - true - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.6 - 1.6 - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.9.1 - - true - - - - attach-javadocs - - jar - - - - - - maven-failsafe-plugin - 2.7 - - - - integration-test - verify - - - - - -Xmx1024m - - - - org.apache.maven.plugins - maven-surefire-plugin - 2.16 - - -Xmx1024m - - - - org.apache.maven.plugins - maven-eclipse-plugin - 2.8 - - - - org.eclipse.jdt.launching.JRE_CONTAINER - - - true - true - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.3.1 - - - enforce-maven - - enforce - - - - - - [3.0.3,) - - - - - - - - org.apache.maven.plugins - maven-source-plugin - 2.2.1 - - - attach-sources - - jar - - - - - - - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - pl.project13.maven - - - git-commit-id-plugin - - - [2.1.5,) - - - revision - - - - - - - - - - org.apache.maven.plugins - - - maven-checkstyle-plugin - - - [2.12.1,) - - - check - - - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - [1.0.0,) - - enforce - - - - - - - - - org.apache.maven.plugins - maven-remote-resources-plugin - [0.0.0,) - - process - - - - - - - - - - - - - - From 94687a64ca87763e5e5339fcb1c33c072d2cc096 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 30 Jul 2014 16:48:48 +0200 Subject: [PATCH 10/14] added missing licenses --- .../lambdas/FilterITCase.java | 1 - .../lambdas/ReduceITCase.java | 21 ++++++++++++++++--- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java index b6c13dcb84714..c7754251921bb 100644 --- a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/FilterITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java index b63f26690e098..ab27fe41327b1 100644 --- a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java +++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/ReduceITCase.java @@ -1,10 +1,25 @@ +/** + * 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.test.javaApiOperators.lambdas; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.ReduceFunction; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.typeutils.BasicTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; From ba38b9032923f42a215ce09d6ceee34a29ce2347 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 30 Jul 2014 17:21:52 +0200 Subject: [PATCH 11/14] removed dummy emacs cache pom file --- flink-java8-tests/pom.xml~ | 238 ------------------------------------- 1 file changed, 238 deletions(-) delete mode 100644 flink-java8-tests/pom.xml~ diff --git a/flink-java8-tests/pom.xml~ b/flink-java8-tests/pom.xml~ deleted file mode 100644 index 5ddb54c24c836..0000000000000 --- a/flink-java8-tests/pom.xml~ +++ /dev/null @@ -1,238 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-tests - 0.6-incubating-SNAPSHOT - .. - - - flink-java8-tests - flink-java8-tests - - jar - - - - org.apache.flink - flink-core - ${project.version} - - - org.apache.flink - flink-compiler - ${project.version} - - - org.apache.flink - flink-runtime - ${project.version} - - - org.apache.flink - flink-clients - ${project.version} - - - org.apache.flink - flink-java - ${project.version} - - - junit - junit - 4.7 - - - org.apache.flink - flink-scala - ${project.version} - test - - - org.apache.flink - flink-test-utils - ${project.version} - test - - - org.apache.flink - flink-java-examples - ${project.version} - test - - - org.apache.flink - flink-scala-examples - ${project.version} - test - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - WARN - - - **/*TestBase*.class - - once - -Xmx1024m - - - - maven-failsafe-plugin - - - WARN - - always - 1 - false - - - - - maven-assembly-plugin - 2.4 - - - create-test-dependency - process-test-classes - - single - - - - - org.apache.flink.test.util.testjar.KMeansForTest - - - maven - false - - src/test/assembly/test-assembly.xml - - - - - - - - - maven-clean-plugin - 2.5 - - - remove-kmeansfortest - process-test-classes - - clean - - - true - - - ${project.build.testOutputDirectory} - - **/testjar/*.class - - - - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - - - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - org.apache.maven.plugins - - - maven-assembly-plugin - - - [2.4,) - - - single - - - - - - - - - - - - - - From 2e1d2b66b8226fd98139b658829a84af20fdf486 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Thu, 31 Jul 2014 09:31:48 +0200 Subject: [PATCH 12/14] Fixed java8-tests pom file, lambda detection using SerializedLambda --- .../common/functions/util/FunctionUtils.java | 24 ++++++-- .../org/apache/flink/api/java/DataSet.java | 6 +- .../api/java/operators/CoGroupOperator.java | 2 +- .../api/java/operators/CrossOperator.java | 2 +- .../api/java/operators/JoinOperator.java | 4 +- .../api/java/operators/SortedGrouping.java | 2 +- .../api/java/operators/UnsortedGrouping.java | 2 +- flink-java8-tests/pom.xml | 61 ++----------------- 8 files changed, 32 insertions(+), 71 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java index 1b14ef6ee5dee..bc4ffd0f0034c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java @@ -24,12 +24,11 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import java.util.regex.Pattern; +import java.lang.invoke.SerializedLambda; +import java.lang.reflect.Method; public class FunctionUtils { - private static final Pattern lambdaPattern = Pattern.compile("(\\S+)\\$\\$Lambda\\$(\\d+)/\\d+"); - public static void openFunction (Function function, Configuration parameters) throws Exception{ if (function instanceof RichFunction) { @@ -62,8 +61,21 @@ public static RuntimeContext getFunctionRuntimeContext (Function function, Runti } } - public static boolean isLambdaFunction (Function function) { - - return lambdaPattern.matcher(function.getClass().getName()).matches(); + public static boolean isSerializedLambdaFunction(Function function) { + Class clazz = function.getClass(); + try { + Method replaceMethod = clazz.getDeclaredMethod("writeReplace"); + replaceMethod.setAccessible(true); + Object serializedForm = replaceMethod.invoke(function); + if (serializedForm instanceof SerializedLambda) { + return true; + } + else { + return false; + } + } + catch (Exception e) { + return false; + } } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 092c48cb120f7..b83c695e1673d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -141,7 +141,7 @@ public MapOperator map(Mappable mapper) { if (mapper == null) { throw new NullPointerException("Map function must not be null."); } - if (FunctionUtils.isLambdaFunction(mapper)) { + if (FunctionUtils.isSerializedLambdaFunction(mapper)) { throw new UnsupportedLambdaExpressionException(); } return new MapOperator(this, mapper); @@ -163,7 +163,7 @@ public FlatMapOperator flatMap(FlatMappable flatMapper) { if (flatMapper == null) { throw new NullPointerException("FlatMap function must not be null."); } - if (FunctionUtils.isLambdaFunction(flatMapper)) { + if (FunctionUtils.isSerializedLambdaFunction(flatMapper)) { throw new UnsupportedLambdaExpressionException(); } return new FlatMapOperator(this, flatMapper); @@ -309,7 +309,7 @@ public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - if (FunctionUtils.isLambdaFunction(reducer)) { + if (FunctionUtils.isSerializedLambdaFunction(reducer)) { throw new UnsupportedLambdaExpressionException(); } return new GroupReduceOperator(this, reducer); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 09b02c3260971..b9847e85bdb09 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -516,7 +516,7 @@ public CoGroupOperator with(CoGroupable function) { if (function == null) { throw new NullPointerException("CoGroup function must not be null."); } - if (FunctionUtils.isLambdaFunction(function)) { + if (FunctionUtils.isSerializedLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } TypeInformation returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 1df568e2f7cea..93bb72ab69dee 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -134,7 +134,7 @@ public CrossOperator with(Crossable function) { if (function == null) { throw new NullPointerException("Cross function must not be null."); } - if (FunctionUtils.isLambdaFunction(function)) { + if (FunctionUtils.isSerializedLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } TypeInformation returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index ac782a6c37c91..c75a3bb2f52ca 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -467,7 +467,7 @@ public EquiJoin with(FlatJoinable function) { if (function == null) { throw new NullPointerException("Join function must not be null."); } - if (FunctionUtils.isLambdaFunction(function)) { + if (FunctionUtils.isSerializedLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } TypeInformation returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type()); @@ -478,7 +478,7 @@ public EquiJoin with (Joinable function) { if (function == null) { throw new NullPointerException("Join function must not be null."); } - if (FunctionUtils.isLambdaFunction(function)) { + if (FunctionUtils.isSerializedLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } FlatJoinable generatedFunction = new WrappingFlatJoinFunction(function); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 9883cc9f826b0..578641aa29362 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -83,7 +83,7 @@ public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - if (FunctionUtils.isLambdaFunction(reducer)) { + if (FunctionUtils.isSerializedLambdaFunction(reducer)) { throw new UnsupportedLambdaExpressionException(); } return new GroupReduceOperator(this, reducer); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 6af6b6e6eb06c..9985c04cbe12c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -129,7 +129,7 @@ public GroupReduceOperator reduceGroup(GroupReducible reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } - if (FunctionUtils.isLambdaFunction(reducer)) { + if (FunctionUtils.isSerializedLambdaFunction(reducer)) { throw new UnsupportedLambdaExpressionException(); } return new GroupReduceOperator(this, reducer); diff --git a/flink-java8-tests/pom.xml b/flink-java8-tests/pom.xml index edfd1a588fed9..2587776e1de6d 100644 --- a/flink-java8-tests/pom.xml +++ b/flink-java8-tests/pom.xml @@ -41,21 +41,11 @@ under the License. flink-core ${project.version} - - org.apache.flink - flink-compiler - ${project.version} - - - org.apache.flink - flink-runtime - ${project.version} - - - org.apache.flink - flink-clients - ${project.version} - + + org.apache.flink + flink-test-utils + ${project.version} + org.apache.flink flink-java @@ -66,35 +56,6 @@ under the License. junit 4.7 - - org.apache.flink - flink-scala - ${project.version} - test - - - org.apache.flink - flink-test-utils - ${project.version} - test - - - org.apache.flink - flink-java-examples - ${project.version} - test - - - org.apache.flink - flink-scala-examples - ${project.version} - test - - - org.apache.flink - flink-tests - ${project.version} - @@ -143,18 +104,6 @@ under the License. false - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - From dcb54d09f3ff9b3d2ddfd72e0941a819cb03baa2 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Thu, 31 Jul 2014 16:36:59 +0200 Subject: [PATCH 13/14] Renaming of SAMs to *Function and rich functions to Rich*Function, changed examples and tests to use SAMs --- .../{CoGroupable.java => CoGroupFunction.java} | 2 +- ...ProposedNewCombinable.java => CombineFunction.java} | 2 +- .../functions/{Crossable.java => CrossFunction.java} | 2 +- .../functions/{Filterable.java => FilterFunction.java} | 2 +- .../{FlatCombinable.java => FlatCombineFunction.java} | 2 +- .../{FlatJoinable.java => FlatJoinFunction.java} | 2 +- .../{FlatMappable.java => FlatMapFunction.java} | 2 +- .../{GroupReducible.java => GroupReduceFunction.java} | 2 +- .../functions/{Joinable.java => JoinFunction.java} | 2 +- .../functions/{Mappable.java => MapFunction.java} | 2 +- .../functions/{Reducible.java => ReduceFunction.java} | 2 +- .../{CoGroupFunction.java => RichCoGroupFunction.java} | 2 +- .../{CrossFunction.java => RichCrossFunction.java} | 2 +- .../{FilterFunction.java => RichFilterFunction.java} | 2 +- ...mbineFunction.java => RichFlatCombineFunction.java} | 2 +- ...FlatJoinFunction.java => RichFlatJoinFunction.java} | 2 +- .../{FlatMapFunction.java => RichFlatMapFunction.java} | 4 ++-- ...educeFunction.java => RichGroupReduceFunction.java} | 10 +++++----- .../{JoinFunction.java => RichJoinFunction.java} | 2 +- .../{MapFunction.java => RichMapFunction.java} | 2 +- .../{ReduceFunction.java => RichReduceFunction.java} | 2 +- 21 files changed, 26 insertions(+), 26 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{CoGroupable.java => CoGroupFunction.java} (95%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{ProposedNewCombinable.java => CombineFunction.java} (93%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Crossable.java => CrossFunction.java} (94%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Filterable.java => FilterFunction.java} (94%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{FlatCombinable.java => FlatCombineFunction.java} (93%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{FlatJoinable.java => FlatJoinFunction.java} (92%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{FlatMappable.java => FlatMapFunction.java} (95%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{GroupReducible.java => GroupReduceFunction.java} (95%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Joinable.java => JoinFunction.java} (92%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Mappable.java => MapFunction.java} (95%) rename flink-core/src/main/java/org/apache/flink/api/common/functions/{Reducible.java => ReduceFunction.java} (95%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{CoGroupFunction.java => RichCoGroupFunction.java} (96%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{CrossFunction.java => RichCrossFunction.java} (95%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{FilterFunction.java => RichFilterFunction.java} (95%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{CombineFunction.java => RichFlatCombineFunction.java} (91%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{FlatJoinFunction.java => RichFlatJoinFunction.java} (96%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{FlatMapFunction.java => RichFlatMapFunction.java} (93%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{GroupReduceFunction.java => RichGroupReduceFunction.java} (90%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{JoinFunction.java => RichJoinFunction.java} (90%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{MapFunction.java => RichMapFunction.java} (95%) rename flink-java/src/main/java/org/apache/flink/api/java/functions/{ReduceFunction.java => RichReduceFunction.java} (96%) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java index 29f28c361ec18..5c200afddcd7e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java @@ -25,7 +25,7 @@ import org.apache.flink.util.Collector; -public interface CoGroupable extends Function, Serializable { +public interface CoGroupFunction extends Function, Serializable { /** * This method must be implemented to provide a user implementation of a diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CombineFunction.java similarity index 93% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/CombineFunction.java index 92389eb796250..d72c4c8ed376a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/ProposedNewCombinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CombineFunction.java @@ -25,7 +25,7 @@ /** * Generic interface used for combiners. */ -public interface ProposedNewCombinable extends Function, Serializable { +public interface CombineFunction extends Function, Serializable { T combine(Iterator records) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java similarity index 94% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java index 2b56987e30c62..0c8bc97829f62 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Crossable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java @@ -27,7 +27,7 @@ * @param Second input type * @param Output type */ -public interface Crossable extends Function, Serializable { +public interface CrossFunction extends Function, Serializable { /** * User defined function for the cross operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java similarity index 94% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java index ed5f9a15a1def..2f684778ab38c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Filterable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java @@ -20,7 +20,7 @@ package org.apache.flink.api.common.functions; import java.io.Serializable; -public interface Filterable extends Function, Serializable { +public interface FilterFunction extends Function, Serializable { /** * User defined function for a filter. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombineFunction.java similarity index 93% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombineFunction.java index bb600207d00cd..b2c8f3079f4e6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatCombineFunction.java @@ -27,7 +27,7 @@ /** * Generic interface used for combiners. */ -public interface FlatCombinable extends Function, Serializable { +public interface FlatCombineFunction extends Function, Serializable { void combine(Iterator values, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java similarity index 92% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java index b5afb90cbed46..6a6b971b45038 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java @@ -24,7 +24,7 @@ import java.io.Serializable; -public interface FlatJoinable extends Function, Serializable { +public interface FlatJoinFunction extends Function, Serializable { void join (IN1 left, IN2 right, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java index c8721e2e5ce77..a8696cf528ae3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMappable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java @@ -29,7 +29,7 @@ * @param * @param */ -public interface FlatMappable extends Function, Serializable { +public interface FlatMapFunction extends Function, Serializable { /** * The core method of FlatMappable. Takes an element from the input data set and transforms diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java index 0999c5ba2889c..984d1fd1e492c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReducible.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java @@ -30,7 +30,7 @@ * @param Incoming types * @param Outgoing types */ -public interface GroupReducible extends Function, Serializable { +public interface GroupReduceFunction extends Function, Serializable { /** * * The central function to be implemented for a reducer. The function receives per call one diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java similarity index 92% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java index a810386fa9df2..02f526aa970e5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Joinable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java @@ -22,7 +22,7 @@ import java.io.Serializable; -public interface Joinable extends Function, Serializable { +public interface JoinFunction extends Function, Serializable { OUT join(IN1 first, IN2 second) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java index bc80b76122152..4e2520deefe42 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Mappable.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java @@ -22,7 +22,7 @@ import java.io.Serializable; -public interface Mappable extends Function, Serializable { +public interface MapFunction extends Function, Serializable { /** * The core method of Mappable. Takes an element from the input data set and transforms diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java rename to flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java index bcd5f6147ed3a..04f690a1ecc9e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/Reducible.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java @@ -22,7 +22,7 @@ import java.io.Serializable; -public interface Reducible extends Function, Serializable { +public interface ReduceFunction extends Function, Serializable { /** * The core method of Reducible, combining two values into one value of the same type. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java similarity index 96% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java index 0f0fe071df360..3f49314f84c03 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java @@ -50,7 +50,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CoGroupFunction extends AbstractRichFunction implements CoGroupable { +public abstract class RichCoGroupFunction extends AbstractRichFunction implements CoGroupable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java similarity index 95% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java index bc75e196596c8..6a64834086385 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java @@ -45,7 +45,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class CrossFunction extends AbstractRichFunction implements Crossable { +public abstract class RichCrossFunction extends AbstractRichFunction implements Crossable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java similarity index 95% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java index 46617d8afb54f..78d126c8d6f70 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FilterFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java @@ -36,7 +36,7 @@ * * @param The type of the filtered elements. */ -public abstract class FilterFunction extends AbstractRichFunction implements Filterable { +public abstract class RichFilterFunction extends AbstractRichFunction implements Filterable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java similarity index 91% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java index b612aab010e2c..e369d9ad564ab 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CombineFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java @@ -25,7 +25,7 @@ import java.util.Iterator; -public abstract class CombineFunction extends AbstractRichFunction implements FlatCombinable { +public abstract class RichFlatCombineFunction extends AbstractRichFunction implements FlatCombinable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java similarity index 96% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java index 53cbcd43e30b1..1dab5bce48316 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatJoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java @@ -55,7 +55,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class FlatJoinFunction extends AbstractRichFunction implements FlatJoinable { +public abstract class RichFlatJoinFunction extends AbstractRichFunction implements FlatJoinable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java similarity index 93% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java index ab879a9573dd2..02398957160ce 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java @@ -26,7 +26,7 @@ * The abstract base class for flatMap functions. FlatMap functions take elements and transform them, * into zero, one, or more elements. Typical applications can be splitting elements, or unnesting lists * and arrays. Operations that produce multiple strictly one result element per input element can also - * use the {@link MapFunction}. + * use the {@link RichMapFunction}. *

* The basic syntax for using a FlatMapFunction is as follows: *

@@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class FlatMapFunction extends AbstractRichFunction implements FlatMappable { +public abstract class RichFlatMapFunction extends AbstractRichFunction implements FlatMappable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java similarity index 90% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java index c04437a289e2f..2e37780a5245b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java @@ -34,7 +34,7 @@ * They may aggregate them to a single value, or produce multiple result values for each group. *

* For a reduce functions that works incrementally by combining always two elements, see - * {@link ReduceFunction}, called via {@link org.apache.flink.api.java.DataSet#reduce(ReduceFunction)}. + * {@link RichReduceFunction}, called via {@link org.apache.flink.api.java.DataSet#reduce(RichReduceFunction)}. *

* The basic syntax for using a grouped GroupReduceFunction is as follows: *

@@ -51,7 +51,7 @@ * @param Type of the elements that this function processes. * @param The type of the elements returned by the user-defined function. */ -public abstract class GroupReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { +public abstract class RichGroupReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { private static final long serialVersionUID = 1L; @@ -74,7 +74,7 @@ public abstract class GroupReduceFunction extends AbstractRichFunction * to reorganizing the data in an expensive way, as might be required for the final * reduce function. *

- * This method is only ever invoked when the subclass of {@link GroupReduceFunction} + * This method is only ever invoked when the subclass of {@link RichGroupReduceFunction} * adds the {@link Combinable} annotation, or if the combinable flag is set when defining * the reduceGroup operation via * {@link org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean)}. @@ -101,8 +101,8 @@ public void combine(Iterator values, Collector out) throws Exception { // -------------------------------------------------------------------------------------------- /** - * This annotation can be added to classes that extend {@link GroupReduceFunction}, in oder to mark - * them as "combinable". The system may call the {@link GroupReduceFunction#combine(Iterator, Collector)} + * This annotation can be added to classes that extend {@link RichGroupReduceFunction}, in oder to mark + * them as "combinable". The system may call the {@link RichGroupReduceFunction#combine(Iterator, Collector)} * method on such functions, to pre-reduce the data before transferring it over the network to * the actual group reduce operation. *

diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java similarity index 90% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java index 14e5beff81aa8..9c0608720ed89 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.Joinable; -public abstract class JoinFunction extends AbstractRichFunction implements Joinable { +public abstract class RichJoinFunction extends AbstractRichFunction implements Joinable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java similarity index 95% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java index 7a8d2cd044dc3..87485b08dfb8b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/MapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class MapFunction extends AbstractRichFunction implements Mappable { +public abstract class RichMapFunction extends AbstractRichFunction implements Mappable { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java similarity index 96% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java rename to flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java index fd3c24935b1b7..50580cf431e81 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java @@ -44,7 +44,7 @@ * * @param Type of the elements that this function processes. */ -public abstract class ReduceFunction extends AbstractRichFunction implements Reducible { +public abstract class RichReduceFunction extends AbstractRichFunction implements Reducible { private static final long serialVersionUID = 1L; From 4868dd9980003dc45f49c6125d104c72a096d41c Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Thu, 31 Jul 2014 16:37:09 +0200 Subject: [PATCH 14/14] Renaming of SAMs to *Function and rich functions to Rich*Function, changed examples and tests to use SAMs --- .../flink/api/avro/AvroOutputFormatTest.java | 6 +- .../avro/testjar/AvroExternalJarProgram.java | 8 +- .../mapred/example/WordCount.java | 8 +- .../mapreduce/example/WordCount.java | 8 +- .../spargel/java/VertexCentricIteration.java | 8 +- .../examples/SpargelConnectedComponents.java | 4 +- .../java/examples/SpargelPageRank.java | 8 +- .../SpargelPageRankCountingVertices.java | 14 +-- .../SpargelConnectedComponentsITCase.java | 4 +- .../flink/client/testjar/WordCount.java | 4 +- .../compiler/BranchingPlansCompilerTest.java | 5 +- .../CachedMatchStrategyCompilerTest.java | 5 +- .../compiler/CoGroupSolutionSetFirstTest.java | 9 +- .../compiler/FeedbackPropertiesMatchTest.java | 8 +- .../compiler/GroupReduceCompilationTest.java | 14 +-- .../compiler/IterationsCompilerTest.java | 20 ++-- .../flink/compiler/ReduceCompilationTest.java | 10 +- .../UnionPropertyPropagationTest.java | 4 +- .../WorksetIterationsJavaApiCompilerTest.java | 18 +-- .../testfunctions/DummyFlatJoinFunction.java | 4 +- .../testfunctions/IdentityGroupReducer.java | 6 +- .../testfunctions/IdentityMapper.java | 4 +- .../testfunctions/SelectOneReducer.java | 4 +- .../testfunctions/Top1GroupReducer.java | 6 +- .../operators/base/CoGroupOperatorBase.java | 6 +- .../operators/base/CrossOperatorBase.java | 6 +- .../operators/base/FilterOperatorBase.java | 6 +- .../operators/base/FlatMapOperatorBase.java | 6 +- .../base/GroupReduceOperatorBase.java | 14 +-- .../operators/base/JoinOperatorBase.java | 6 +- .../operators/base/MapOperatorBase.java | 4 +- .../operators/base/ReduceOperatorBase.java | 6 +- .../common/operators/util/OperatorUtil.java | 16 +-- .../operators/util/OperatorUtilTest.java | 16 +-- .../flink/example/java/clustering/KMeans.java | 17 +-- .../java/graph/ConnectedComponents.java | 16 +-- .../java/graph/EnumTrianglesBasic.java | 14 +-- .../example/java/graph/EnumTrianglesOpt.java | 28 ++--- .../example/java/graph/PageRankBasic.java | 18 +-- .../java/graph/TransitiveClosureNaive.java | 4 +- .../example/java/ml/LinearRegression.java | 15 +-- .../EmptyFieldsCountAccumulator.java | 4 +- .../java/relational/RelationalQuery.java | 2 +- .../example/java/relational/TPCHQuery10.java | 4 +- .../example/java/relational/TPCHQuery3.java | 4 +- .../java/relational/WebLogAnalysis.java | 12 +- .../example/java/wordcount/WordCount.java | 4 +- .../example/java/wordcount/WordCountPOJO.java | 6 +- .../org/apache/flink/api/java/DataSet.java | 64 +++++------ .../flink/api/java/ExecutionEnvironment.java | 2 +- .../api/java/functions/FlatMapIterator.java | 4 +- .../java/functions/FunctionAnnotation.java | 16 +-- .../java/functions/GroupReduceIterator.java | 2 +- .../java/functions/RichCoGroupFunction.java | 4 +- .../api/java/functions/RichCrossFunction.java | 4 +- .../java/functions/RichFilterFunction.java | 4 +- .../functions/RichFlatCombineFunction.java | 4 +- .../java/functions/RichFlatJoinFunction.java | 6 +- .../java/functions/RichFlatMapFunction.java | 4 +- .../functions/RichGroupReduceFunction.java | 6 +- .../api/java/functions/RichJoinFunction.java | 4 +- .../api/java/functions/RichMapFunction.java | 6 +- .../java/functions/RichReduceFunction.java | 8 +- .../api/java/operators/AggregateOperator.java | 20 ++-- .../api/java/operators/CoGroupOperator.java | 59 +++++----- .../api/java/operators/CrossOperator.java | 20 ++-- .../api/java/operators/DistinctOperator.java | 24 ++-- .../api/java/operators/FilterOperator.java | 10 +- .../api/java/operators/FlatMapOperator.java | 10 +- .../java/operators/GroupReduceOperator.java | 32 +++--- .../flink/api/java/operators/Grouping.java | 4 +- .../api/java/operators/JoinOperator.java | 74 ++++++------ .../flink/api/java/operators/MapOperator.java | 12 +- .../api/java/operators/ProjectOperator.java | 4 +- .../api/java/operators/ReduceOperator.java | 26 ++--- .../operators/SingleInputUdfOperator.java | 4 +- .../api/java/operators/SortedGrouping.java | 11 +- .../java/operators/TwoInputUdfOperator.java | 4 +- .../flink/api/java/operators/UdfOperator.java | 4 +- .../api/java/operators/UnsortedGrouping.java | 18 ++- .../translation/KeyExtractingMapper.java | 4 +- .../translation/KeyRemovingMapper.java | 4 +- .../translation/PlanFilterOperator.java | 14 +-- .../translation/PlanProjectOperator.java | 6 +- .../PlanUnwrappingCoGroupOperator.java | 16 +-- .../PlanUnwrappingJoinOperator.java | 16 +-- .../PlanUnwrappingReduceGroupOperator.java | 26 ++--- .../PlanUnwrappingReduceOperator.java | 12 +- .../translation/TupleKeyExtractingMapper.java | 4 +- .../record/functions/CoGroupFunction.java | 3 +- .../java/record/functions/CrossFunction.java | 3 +- .../java/record/functions/JoinFunction.java | 4 +- .../java/record/functions/ReduceFunction.java | 6 +- .../api/java/typeutils/TypeExtractor.java | 64 +++++------ .../SemanticPropertiesTranslationTest.java | 12 +- .../DeltaIterationTranslationTest.java | 16 +-- .../translation/ReduceTranslationTests.java | 10 +- .../type/extractor/TypeExtractorTest.java | 108 +++++++++--------- .../AbstractCachedBuildSideMatchDriver.java | 6 +- .../operators/AllGroupReduceDriver.java | 24 ++-- .../runtime/operators/AllReduceDriver.java | 16 +-- .../runtime/operators/CoGroupDriver.java | 14 +-- .../CoGroupWithSolutionSetFirstDriver.java | 14 +-- .../CoGroupWithSolutionSetSecondDriver.java | 14 +-- .../flink/runtime/operators/CrossDriver.java | 20 ++-- .../runtime/operators/FlatMapDriver.java | 16 +-- .../operators/GroupReduceCombineDriver.java | 14 +-- .../runtime/operators/GroupReduceDriver.java | 16 +-- .../JoinWithSolutionSetFirstDriver.java | 14 +-- .../JoinWithSolutionSetSecondDriver.java | 14 +-- .../flink/runtime/operators/MapDriver.java | 16 +-- .../flink/runtime/operators/MatchDriver.java | 16 +-- .../operators/ReduceCombineDriver.java | 16 +-- .../flink/runtime/operators/ReduceDriver.java | 16 +-- .../runtime/operators/RegularPactTask.java | 23 ++-- .../chaining/ChainedFlatMapDriver.java | 8 +- .../operators/chaining/ChainedMapDriver.java | 8 +- .../SynchronousChainedCombineDriver.java | 10 +- .../hash/BuildFirstHashMatchIterator.java | 4 +- .../hash/BuildSecondHashMatchIterator.java | 4 +- .../sort/CombiningUnilateralSortMerger.java | 12 +- .../operators/sort/MergeMatchIterator.java | 10 +- .../operators/util/JoinTaskIterator.java | 4 +- .../operators/CachedMatchTaskTest.java | 7 +- .../operators/CoGroupTaskExternalITCase.java | 7 +- .../runtime/operators/CoGroupTaskTest.java | 9 +- .../operators/CombineTaskExternalITCase.java | 4 +- .../runtime/operators/CombineTaskTest.java | 4 +- .../operators/CrossTaskExternalITCase.java | 4 +- .../runtime/operators/CrossTaskTest.java | 10 +- .../operators/MatchTaskExternalITCase.java | 4 +- .../runtime/operators/MatchTaskTest.java | 4 +- .../operators/ReduceTaskExternalITCase.java | 4 +- .../runtime/operators/ReduceTaskTest.java | 4 +- .../drivers/AllGroupReduceDriverTest.java | 20 ++-- .../drivers/AllReduceDriverTest.java | 32 +++--- .../drivers/GroupReduceDriverTest.java | 20 ++-- .../drivers/ReduceCombineDriverTest.java | 32 +++--- .../operators/drivers/ReduceDriverTest.java | 32 +++--- .../hash/HashMatchIteratorITCase.java | 8 +- ...tipleJoinsWithSolutionSetCompilerTest.java | 14 +-- .../BulkIterationWithAllReducerITCase.java | 4 +- ...oGroupConnectedComponentsSecondITCase.java | 12 +- .../DependencyConnectedComponentsITCase.java | 23 ++-- .../aggregators/AggregatorsITCase.java | 20 ++-- ...ntsWithParametrizableAggregatorITCase.java | 15 +-- ...tsWithParametrizableConvergenceITCase.java | 15 +-- .../CustomCompensatableDotProductCoGroup.java | 4 +- .../CustomCompensatableDotProductMatch.java | 4 +- .../CustomRankCombiner.java | 8 +- .../test/javaApiOperators/CoGroupITCase.java | 17 +-- .../test/javaApiOperators/CrossITCase.java | 52 ++------- .../test/javaApiOperators/DistinctITCase.java | 4 +- .../test/javaApiOperators/FilterITCase.java | 33 +----- .../test/javaApiOperators/FlatMapITCase.java | 47 +------- .../javaApiOperators/GroupReduceITCase.java | 79 +++---------- .../test/javaApiOperators/JoinITCase.java | 18 +-- .../test/javaApiOperators/MapITCase.java | 43 +------ .../test/javaApiOperators/ReduceITCase.java | 56 ++------- .../test/javaApiOperators/UnionITCase.java | 4 +- .../test/util/testjar/KMeansForTest.java | 16 +-- 161 files changed, 1007 insertions(+), 1240 deletions(-) diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java index 637a5e953c0cd..386f318d37b21 100644 --- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java +++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java @@ -30,7 +30,7 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.io.AvroOutputFormat; import org.apache.flink.api.java.record.io.avro.example.User; import org.apache.flink.api.java.tuple.Tuple3; @@ -125,7 +125,7 @@ protected void postSubmit() throws Exception { } - public final static class ConvertToUser extends MapFunction, User> { + public final static class ConvertToUser extends RichMapFunction, User> { @Override public User map(Tuple3 value) throws Exception { @@ -133,7 +133,7 @@ public User map(Tuple3 value) throws Exception { } } - public final static class ConvertToReflective extends MapFunction { + public final static class ConvertToReflective extends RichMapFunction { @Override public ReflectiveUser map(User value) throws Exception { diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java index 146c72bc7499d..75b7da6933657 100644 --- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java +++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java @@ -40,8 +40,8 @@ import org.apache.avro.reflect.ReflectData; import org.apache.avro.reflect.ReflectDatumWriter; import org.apache.flink.api.avro.AvroBaseValue; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichMapFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -141,7 +141,7 @@ public SUser(MyUser u) { // -------------------------------------------------------------------------------------------- - public static final class NameExtractor extends MapFunction> { + public static final class NameExtractor extends RichMapFunction> { private static final long serialVersionUID = 1L; @Override @@ -151,7 +151,7 @@ public Tuple2 map(MyUser u) { } } - public static final class NameGrouper extends ReduceFunction> { + public static final class NameGrouper extends RichReduceFunction> { private static final long serialVersionUID = 1L; @Override diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java index 4e8ffa970f664..ba09e77b95df4 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java @@ -18,6 +18,7 @@ package org.apache.flink.hadoopcompatibility.mapred.example; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; @@ -28,8 +29,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.hadoopcompatibility.mapred.HadoopInputFormat; import org.apache.flink.hadoopcompatibility.mapred.HadoopOutputFormat; @@ -88,7 +88,7 @@ public static void main(String[] args) throws Exception { /** * Splits a line into words and converts Hadoop Writables into normal Java data types. */ - public static final class Tokenizer extends FlatMapFunction, Tuple2> { + public static final class Tokenizer extends RichFlatMapFunction, Tuple2> { @Override public void flatMap(Tuple2 value, Collector> out) { @@ -108,7 +108,7 @@ public void flatMap(Tuple2 value, Collector, Tuple2> { + public static final class HadoopDatatypeMapper extends RichMapFunction, Tuple2> { @Override public Tuple2 map(Tuple2 value) throws Exception { diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java index 36ea378226235..c00a14ae2f0bb 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java @@ -19,8 +19,8 @@ package org.apache.flink.hadoopcompatibility.mapreduce.example; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; import org.apache.hadoop.fs.Path; @@ -89,7 +89,7 @@ public static void main(String[] args) throws Exception { /** * Splits a line into words and converts Hadoop Writables into normal Java data types. */ - public static final class Tokenizer extends FlatMapFunction, Tuple2> { + public static final class Tokenizer extends RichFlatMapFunction, Tuple2> { @Override public void flatMap(Tuple2 value, Collector> out) { @@ -109,7 +109,7 @@ public void flatMap(Tuple2 value, Collector, Tuple2> { + public static final class HadoopDatatypeMapper extends RichMapFunction, Tuple2> { @Override public Tuple2 map(Tuple2 value) throws Exception { diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java index bb84ceaa5282a..65be2f8b6f133 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java @@ -29,7 +29,7 @@ import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration; -import org.apache.flink.api.java.functions.CoGroupFunction; +import org.apache.flink.api.java.functions.RichCoGroupFunction; import org.apache.flink.api.java.operators.CoGroupOperator; import org.apache.flink.api.java.operators.CustomUnaryOperation; import org.apache.flink.api.java.tuple.Tuple2; @@ -393,7 +393,7 @@ VertexCentricIteration withValuedEdg // -------------------------------------------------------------------------------------------- private static final class VertexUpdateUdf, VertexValue, Message> - extends CoGroupFunction, Tuple2, Tuple2> + extends RichCoGroupFunction, Tuple2, Tuple2> implements ResultTypeQueryable> { private static final long serialVersionUID = 1L; @@ -463,7 +463,7 @@ public TypeInformation> getProducedType() { * UDF that encapsulates the message sending function for graphs where the edges have no associated values. */ private static final class MessagingUdfNoEdgeValues, VertexValue, Message> - extends CoGroupFunction, Tuple2, Tuple2> + extends RichCoGroupFunction, Tuple2, Tuple2> implements ResultTypeQueryable> { private static final long serialVersionUID = 1L; @@ -516,7 +516,7 @@ public TypeInformation> getProducedType() { * UDF that encapsulates the message sending function for graphs where the edges have an associated value. */ private static final class MessagingUdfWithEdgeValues, VertexValue, Message, EdgeValue> - extends CoGroupFunction, Tuple2, Tuple2> + extends RichCoGroupFunction, Tuple2, Tuple2> implements ResultTypeQueryable> { private static final long serialVersionUID = 1L; diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelConnectedComponents.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelConnectedComponents.java index ea90feb60381b..a4ba6fa78cda0 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelConnectedComponents.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelConnectedComponents.java @@ -18,7 +18,7 @@ package org.apache.flink.spargel.java.examples; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.spargel.java.MessageIterator; import org.apache.flink.spargel.java.MessagingFunction; @@ -70,7 +70,7 @@ public void sendMessages(Long vertexId, Long componentId) { * A map function that takes a Long value and creates a 2-tuple out of it: *

(Long value) -> (value, value)
*/ - public static final class IdAssigner extends MapFunction> { + public static final class IdAssigner extends RichMapFunction> { @Override public Tuple2 map(Long value) { return new Tuple2(value, value); diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRank.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRank.java index c7fbaaa191059..9dfc32773aaf5 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRank.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRank.java @@ -18,8 +18,8 @@ package org.apache.flink.spargel.java.examples; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.spargel.java.MessageIterator; @@ -48,7 +48,7 @@ public static void main(String[] args) throws Exception { // enumerate some sample edges and assign an initial uniform probability (rank) DataSet> intialRanks = env.generateSequence(1, numVertices) - .map(new MapFunction>() { + .map(new RichMapFunction>() { public Tuple2 map(Long value) { return new Tuple2(value, 1.0/numVertices); } @@ -56,7 +56,7 @@ public Tuple2 map(Long value) { // generate some random edges. the transition probability on each edge is 1/num-out-edges of the source vertex DataSet> edgesWithProbability = env.generateSequence(1, numVertices) - .flatMap(new FlatMapFunction>() { + .flatMap(new RichFlatMapFunction>() { public void flatMap(Long value, Collector> out) { int numOutEdges = (int) (Math.random() * (numVertices / 2)); for (int i = 0; i < numOutEdges; i++) { diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java index 34c9ad8e2f74d..43c0b8436cadf 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java @@ -18,9 +18,9 @@ package org.apache.flink.spargel.java.examples; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -53,7 +53,7 @@ public static void main(String[] args) throws Exception { // generate some random edges. the transition probability on each edge is 1/num-out-edges of the source vertex DataSet> edgesWithProbability = env.generateSequence(1, NUM_VERTICES) - .flatMap(new FlatMapFunction>() { + .flatMap(new RichFlatMapFunction>() { public void flatMap(Long value, Collector> out) { int numOutEdges = (int) (Math.random() * (NUM_VERTICES / 2)); for (int i = 0; i < numOutEdges; i++) { @@ -67,12 +67,12 @@ public void flatMap(Long value, Collector> out) { // count the number of vertices DataSet count = vertices - .map(new MapFunction() { + .map(new RichMapFunction() { public Long map(Long value) { return 1L; } }) - .reduce(new ReduceFunction() { + .reduce(new RichReduceFunction() { public Long reduce(Long value1, Long value2) { return value1 + value2; } @@ -80,7 +80,7 @@ public Long reduce(Long value1, Long value2) { // enumerate some sample edges and assign an initial uniform probability (rank) DataSet> intialRanks = vertices - .map(new MapFunction>() { + .map(new RichMapFunction>() { private long numVertices; diff --git a/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java b/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java index a34f2dbcdb56d..16b004c6b9fb1 100644 --- a/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java +++ b/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java @@ -21,7 +21,7 @@ import java.io.BufferedReader; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -72,7 +72,7 @@ protected void postSubmit() throws Exception { } } - public static final class EdgeParser extends MapFunction> { + public static final class EdgeParser extends RichMapFunction> { public Tuple2 map(String value) { String[] nums = value.split(" "); return new Tuple2(Long.parseLong(nums[0]), Long.parseLong(nums[1])); diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java index 7320b7b85a846..2b64b84f06e5e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java +++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java @@ -19,7 +19,7 @@ package org.apache.flink.client.testjar; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; @@ -75,7 +75,7 @@ public static void main(String[] args) throws Exception { * FlatMapFunction. The function takes a line (String) and splits it into * multiple pairs in the form of "(word,1)" (Tuple2). */ - public static final class Tokenizer extends FlatMapFunction> { + public static final class Tokenizer extends RichFlatMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java index 571f4e4840972..10fa34fdae802 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java @@ -30,7 +30,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; import org.apache.flink.api.java.record.operators.BulkIteration; import org.apache.flink.api.java.record.operators.CoGroupOperator; import org.apache.flink.api.java.record.operators.CrossOperator; @@ -40,7 +40,6 @@ import org.apache.flink.api.java.record.operators.JoinOperator; import org.apache.flink.api.java.record.operators.MapOperator; import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.compiler.PactCompiler; import org.apache.flink.compiler.plan.OptimizedPlan; import org.apache.flink.compiler.plan.SinkPlanNode; import org.apache.flink.compiler.plantranslate.NepheleJobGraphGenerator; @@ -959,7 +958,7 @@ public void testBranchingBroadcastVariable() { result1.join(result2) .where(new IdentityKeyExtractor()) .equalTo(new IdentityKeyExtractor()) - .with(new JoinFunction() { + .with(new RichJoinFunction() { @Override public String join(String first, String second) { return null; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java index 55c53f04fec79..8226dbfba0dc4 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java @@ -28,9 +28,8 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.compiler.PactCompiler; import org.apache.flink.compiler.dag.TempMode; import org.apache.flink.compiler.plan.DualInputPlanNode; import org.apache.flink.compiler.plan.OptimizedPlan; @@ -256,7 +255,7 @@ private Plan getTestPlanLeftStatic(String strategy) { } - private static class DummyJoiner extends JoinFunction, Tuple3, Tuple3> { + private static class DummyJoiner extends RichJoinFunction, Tuple3, Tuple3> { @Override public Tuple3 join(Tuple3 first, diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java index 1c30545d07e4c..3624d8653c814 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java @@ -21,16 +21,15 @@ import java.util.Iterator; +import org.apache.flink.api.java.functions.RichCoGroupFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.junit.Assert; import org.junit.Test; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.compiler.CompilerException; import org.apache.flink.compiler.plan.Channel; import org.apache.flink.compiler.plan.DualInputPlanNode; import org.apache.flink.compiler.plan.OptimizedPlan; @@ -43,13 +42,13 @@ @SuppressWarnings("serial") public class CoGroupSolutionSetFirstTest extends CompilerTestBase { - public static class SimpleCGroup extends CoGroupFunction, Tuple1, Tuple1> { + public static class SimpleCGroup extends RichCoGroupFunction, Tuple1, Tuple1> { @Override public void coGroup(Iterator> first, Iterator> second, Collector> out) throws Exception { } } - public static class SimpleMap extends MapFunction, Tuple1> { + public static class SimpleMap extends RichMapFunction, Tuple1> { @Override public Tuple1 map(Tuple1 value) throws Exception { return null; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java index b925ff2524790..0fbf07200da0f 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java @@ -22,8 +22,8 @@ import static org.apache.flink.compiler.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport.*; import static org.junit.Assert.*; -import org.apache.flink.api.common.functions.FlatJoinable; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.OperatorInformation; import org.apache.flink.api.common.operators.Order; @@ -1426,10 +1426,10 @@ private static final DataSourceNode getSourceNode() { } private static final MapNode getMapNode() { - return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); + return new MapNode(new MapOperatorBase>(new IdentityMapper(), new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op")); } private static final MatchNode getJoinNode() { - return new MatchNode(new JoinOperatorBase>(new DummyFlatJoinFunction(), new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op")); + return new MatchNode(new JoinOperatorBase>(new DummyFlatJoinFunction(), new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op")); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java index 8a76471c8401d..b9cc769b23abd 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.tuple.Tuple2; @@ -50,7 +50,7 @@ public void testAllGroupReduceNoCombiner() { DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source"); - data.reduceGroup(new GroupReduceFunction() { + data.reduceGroup(new RichGroupReduceFunction() { public void reduce(Iterator values, Collector out) {} }).name("reducer") .print().name("sink"); @@ -94,7 +94,7 @@ public void testAllReduceWithCombiner() { DataSet data = env.generateSequence(1, 8000000).name("source"); - GroupReduceOperator reduced = data.reduceGroup(new GroupReduceFunction() { + GroupReduceOperator reduced = data.reduceGroup(new RichGroupReduceFunction() { public void reduce(Iterator values, Collector out) {} }).name("reducer"); @@ -147,7 +147,7 @@ public void testGroupedReduceWithFieldPositionKeyNonCombinable() { data .groupBy(1) - .reduceGroup(new GroupReduceFunction, Tuple2>() { + .reduceGroup(new RichGroupReduceFunction, Tuple2>() { public void reduce(Iterator> values, Collector> out) {} }).name("reducer") .print().name("sink"); @@ -196,7 +196,7 @@ public void testGroupedReduceWithFieldPositionKeyCombinable() { GroupReduceOperator, Tuple2> reduced = data .groupBy(1) - .reduceGroup(new GroupReduceFunction, Tuple2>() { + .reduceGroup(new RichGroupReduceFunction, Tuple2>() { public void reduce(Iterator> values, Collector> out) {} }).name("reducer"); @@ -255,7 +255,7 @@ public void testGroupedReduceWithSelectorFunctionKeyNoncombinable() { .groupBy(new KeySelector, String>() { public String getKey(Tuple2 value) { return value.f0; } }) - .reduceGroup(new GroupReduceFunction, Tuple2>() { + .reduceGroup(new RichGroupReduceFunction, Tuple2>() { public void reduce(Iterator> values, Collector> out) {} }).name("reducer") .print().name("sink"); @@ -313,7 +313,7 @@ public void testGroupedReduceWithSelectorFunctionKeyCombinable() { .groupBy(new KeySelector, String>() { public String getKey(Tuple2 value) { return value.f0; } }) - .reduceGroup(new GroupReduceFunction, Tuple2>() { + .reduceGroup(new RichGroupReduceFunction, Tuple2>() { public void reduce(Iterator> values, Collector> out) {} }).name("reducer"); diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java index d3c3e3f643c20..8fc43248159fb 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.*; +import org.apache.flink.api.java.functions.RichFlatMapFunction; import org.junit.Test; import java.util.Iterator; @@ -31,10 +32,9 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; @@ -261,7 +261,7 @@ public static DataSet> doDeltaIteration(DataSet, Tuple2, Tuple2> { + public static final class Join222 extends RichJoinFunction, Tuple2, Tuple2> { @Override public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) { @@ -269,13 +269,13 @@ public Tuple2 join(Tuple2 vertexWithComponent, Tuple2, Tuple2>, Tuple2> { + public static final class FlatMapJoin extends RichFlatMapFunction, Tuple2>, Tuple2> { @Override public void flatMap(Tuple2, Tuple2> value, Collector> out) {} } - public static final class DummyMap extends MapFunction, Tuple2> { + public static final class DummyMap extends RichMapFunction, Tuple2> { @Override public Tuple2 map(Tuple2 value) throws Exception { @@ -284,14 +284,14 @@ public Tuple2 map(Tuple2 value) throws Exception { } @ConstantFields("0") - public static final class Reduce101 extends GroupReduceFunction, Tuple1> { + public static final class Reduce101 extends RichGroupReduceFunction, Tuple1> { @Override public void reduce(Iterator> values, Collector> out) {} } @ConstantFields("0") - public static final class DuplicateValue extends MapFunction, Tuple2> { + public static final class DuplicateValue extends RichMapFunction, Tuple2> { @Override public Tuple2 map(Tuple1 value) throws Exception { @@ -299,7 +299,7 @@ public Tuple2 map(Tuple1 value) throws Exception { } } - public static final class DuplicateValueScalar extends MapFunction> { + public static final class DuplicateValueScalar extends RichMapFunction> { @Override public Tuple2 map(T value) { diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java index a654872821606..fb8ae8d5fd85f 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.util.FieldList; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.junit.Test; import org.apache.flink.api.java.DataSet; @@ -46,7 +46,7 @@ public void testAllReduceNoCombiner() { DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source"); - data.reduce(new ReduceFunction() { + data.reduce(new RichReduceFunction() { @Override public Double reduce(Double value1, Double value2){ @@ -91,7 +91,7 @@ public void testAllReduceWithCombiner() { DataSet data = env.generateSequence(1, 8000000).name("source"); - data.reduce(new ReduceFunction() { + data.reduce(new RichReduceFunction() { @Override public Long reduce(Long value1, Long value2){ @@ -145,7 +145,7 @@ public void testGroupedReduceWithFieldPositionKey() { data .groupBy(1) - .reduce(new ReduceFunction>() { + .reduce(new RichReduceFunction>() { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2){ return null; @@ -205,7 +205,7 @@ public void testGroupedReduceWithSelectorFunctionKey() { .groupBy(new KeySelector, String>() { public String getKey(Tuple2 value) { return value.f0; } }) - .reduce(new ReduceFunction>() { + .reduce(new RichReduceFunction>() { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2){ return null; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java index 1f653c05d0013..1020c8b894a13 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; import org.apache.flink.api.java.record.operators.FileDataSink; import org.apache.flink.api.java.record.operators.FileDataSource; import org.apache.flink.api.java.record.operators.ReduceOperator; @@ -174,7 +174,7 @@ public void postVisit(PlanNode visitable) { }); } - public static final class DummyFlatMap extends FlatMapFunction> { + public static final class DummyFlatMap extends RichFlatMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java index e04256ced92f7..64a4791955d83 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java @@ -31,9 +31,9 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.compiler.plan.DualInputPlanNode; import org.apache.flink.compiler.plan.OptimizedPlan; @@ -214,7 +214,7 @@ public void testRejectPlanIfSolutionSetKeysAndJoinKeysDontMatch() { iter.getWorkset().join(invariantInput) .where(1, 2) .equalTo(1, 2) - .with(new JoinFunction, Tuple3, Tuple3>() { + .with(new RichJoinFunction, Tuple3, Tuple3>() { public Tuple3 join(Tuple3 first, Tuple3 second) { return first; } @@ -224,7 +224,7 @@ public Tuple3 join(Tuple3 first, Tuple3, Tuple3, Tuple3>() { + .with(new RichJoinFunction, Tuple3, Tuple3>() { public Tuple3 join(Tuple3 first, Tuple3 second) { return second; } @@ -263,7 +263,7 @@ private Plan getJavaTestPlan(boolean joinPreservesSolutionSet, boolean mapBefore iter.getWorkset().join(invariantInput) .where(1, 2) .equalTo(1, 2) - .with(new JoinFunction, Tuple3, Tuple3>() { + .with(new RichJoinFunction, Tuple3, Tuple3>() { public Tuple3 join(Tuple3 first, Tuple3 second) { return first; } @@ -273,7 +273,7 @@ public Tuple3 join(Tuple3 first, Tuple3, Tuple3, Tuple3>() { + .with(new RichJoinFunction, Tuple3, Tuple3>() { public Tuple3 join(Tuple3 first, Tuple3 second) { return second; } @@ -282,7 +282,7 @@ public Tuple3 join(Tuple3 first, Tuple30", "1->1", "2->2" } : null); DataSet> nextWorkset = joinedWithSolutionSet.groupBy(1, 2) - .reduceGroup(new GroupReduceFunction, Tuple3>() { + .reduceGroup(new RichGroupReduceFunction, Tuple3>() { public void reduce(Iterator> values, Collector> out) {} }) .name(NEXT_WORKSET_REDUCER_NAME) @@ -290,7 +290,7 @@ public void reduce(Iterator> values, Collector> nextSolutionSet = mapBeforeSolutionDelta ? - joinedWithSolutionSet.map(new MapFunction,Tuple3>() { public Tuple3 map(Tuple3 value) { return value; } }) + joinedWithSolutionSet.map(new RichMapFunction,Tuple3>() { public Tuple3 map(Tuple3 value) { return value; } }) .name(SOLUTION_DELTA_MAPPER_NAME).withConstantSet("0->0","1->1","2->2") : joinedWithSolutionSet; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java index 0211cab6991aa..2388db464c673 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java @@ -19,10 +19,10 @@ package org.apache.flink.compiler.testfunctions; -import org.apache.flink.api.java.functions.FlatJoinFunction; +import org.apache.flink.api.java.functions.RichFlatJoinFunction; import org.apache.flink.util.Collector; -public class DummyFlatJoinFunction extends FlatJoinFunction { +public class DummyFlatJoinFunction extends RichFlatJoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java index fe61f25dfa480..42275afff8025 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java @@ -21,13 +21,13 @@ import java.util.Iterator; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable; import org.apache.flink.util.Collector; @Combinable -public class IdentityGroupReducer extends GroupReduceFunction { +public class IdentityGroupReducer extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java index b6aa40b7172b9..29fc2c8e2316f 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java @@ -19,9 +19,9 @@ package org.apache.flink.compiler.testfunctions; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; -public class IdentityMapper extends MapFunction { +public class IdentityMapper extends RichMapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java index 91634cc269bc9..7ce267f597dd2 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java @@ -18,9 +18,9 @@ package org.apache.flink.compiler.testfunctions; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; -public class SelectOneReducer extends ReduceFunction { +public class SelectOneReducer extends RichReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java index 26db00e00b22d..3f24e65dc8d81 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java @@ -21,13 +21,13 @@ import java.util.Iterator; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable; import org.apache.flink.util.Collector; @Combinable -public class Top1GroupReducer extends GroupReduceFunction { +public class Top1GroupReducer extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java index a2414c583ac2f..4b85a31371f96 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.Ordering; @@ -28,9 +28,9 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** - * @see org.apache.flink.api.common.functions.CoGroupable + * @see org.apache.flink.api.common.functions.CoGroupFunction */ -public class CoGroupOperatorBase> extends DualInputOperator { +public class CoGroupOperatorBase> extends DualInputOperator { /** * The ordering for the order inside a group from input one. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java index fbea862f0633e..a66ea722fc174 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see org.apache.flink.api.common.functions.Crossable + * @see org.apache.flink.api.common.functions.CrossFunction */ -public class CrossOperatorBase> extends DualInputOperator { +public class CrossOperatorBase> extends DualInputOperator { public CrossOperatorBase(UserCodeWrapper udf, BinaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java index 8dfe215ce42bc..3c28c43ef4781 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see org.apache.flink.api.common.functions.FlatMappable + * @see org.apache.flink.api.common.functions.FlatMapFunction */ -public class FilterOperatorBase> extends SingleInputOperator { +public class FilterOperatorBase> extends SingleInputOperator { public FilterOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java index 8c1f96b463a59..89575b6824400 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -28,9 +28,9 @@ /** - * @see org.apache.flink.api.common.functions.FlatMappable + * @see org.apache.flink.api.common.functions.FlatMapFunction */ -public class FlatMapOperatorBase> extends SingleInputOperator { +public class FlatMapOperatorBase> extends SingleInputOperator { public FlatMapOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java index 13aa684621128..ac5548936c7ea 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java @@ -20,8 +20,8 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -31,9 +31,9 @@ /** - * @see org.apache.flink.api.common.functions.GroupReducible + * @see org.apache.flink.api.common.functions.GroupReduceFunction */ -public class GroupReduceOperatorBase> extends SingleInputOperator { +public class GroupReduceOperatorBase> extends SingleInputOperator { /** * The ordering for the order inside a reduce group. @@ -92,15 +92,15 @@ public Ordering getGroupOrder() { /** * Marks the group reduce operation as combinable. Combinable operations may pre-reduce the * data before the actual group reduce operations. Combinable user-defined functions - * must implement the interface {@link org.apache.flink.api.common.functions.FlatCombinable}. + * must implement the interface {@link org.apache.flink.api.common.functions.FlatCombineFunction}. * * @param combinable Flag to mark the group reduce operation as combinable. */ public void setCombinable(boolean combinable) { // sanity check - if (combinable && !FlatCombinable.class.isAssignableFrom(this.userFunction.getUserCodeClass())) { + if (combinable && !FlatCombineFunction.class.isAssignableFrom(this.userFunction.getUserCodeClass())) { throw new IllegalArgumentException("Cannot set a UDF as combinable if it does not implement the interface " + - FlatCombinable.class.getName()); + FlatCombineFunction.class.getName()); } else { this.combinable = combinable; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java index 3fbdefa257e60..2ce052940622d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -27,9 +27,9 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper; /** - * @see org.apache.flink.api.common.functions.FlatJoinable + * @see org.apache.flink.api.common.functions.FlatJoinFunction */ -public class JoinOperatorBase> extends DualInputOperator +public class JoinOperatorBase> extends DualInputOperator { public JoinOperatorBase(UserCodeWrapper udf, BinaryOperatorInformation operatorInfo, int[] keyPositions1, int[] keyPositions2, String name) { super(udf, operatorInfo, keyPositions1, keyPositions2, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java index 92be186f2f99b..26fde056cad57 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -33,7 +33,7 @@ * @param The result type. * @param The type of the user-defined function. */ -public class MapOperatorBase> extends SingleInputOperator { +public class MapOperatorBase> extends SingleInputOperator { public MapOperatorBase(UserCodeWrapper udf, UnaryOperatorInformation operatorInfo, String name) { super(udf, operatorInfo, name); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java index f82e6c42d1cb5..e6c435f5c83ba 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.operators.base; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -31,12 +31,12 @@ * Base data flow operator for Reduce user-defined functions. Accepts reduce functions * and key positions. The key positions are expected in the flattened common data model. * - * @see org.apache.flink.api.common.functions.Reducible + * @see org.apache.flink.api.common.functions.ReduceFunction * * @param The type (parameters and return type) of the reduce function. * @param The type of the reduce function. */ -public class ReduceOperatorBase> extends SingleInputOperator { +public class ReduceOperatorBase> extends SingleInputOperator { /** * Creates a grouped reduce data flow operator. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java index b420566c3bdc3..7d6495b07c525 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorUtil.java @@ -25,11 +25,11 @@ import java.util.Map; import java.util.Map.Entry; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.common.functions.Crossable; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.InputFormat; @@ -56,10 +56,10 @@ public class OperatorUtil { static { STUB_CONTRACTS.put(GenericCollectorMap.class, CollectorMapOperatorBase.class); - STUB_CONTRACTS.put(GroupReducible.class, GroupReduceOperatorBase.class); - STUB_CONTRACTS.put(CoGroupable.class, CoGroupOperatorBase.class); - STUB_CONTRACTS.put(Crossable.class, CrossOperatorBase.class); - STUB_CONTRACTS.put(FlatJoinable.class, JoinOperatorBase.class); + STUB_CONTRACTS.put(GroupReduceFunction.class, GroupReduceOperatorBase.class); + STUB_CONTRACTS.put(CoGroupFunction.class, CoGroupOperatorBase.class); + STUB_CONTRACTS.put(CrossFunction.class, CrossOperatorBase.class); + STUB_CONTRACTS.put(FlatJoinFunction.class, JoinOperatorBase.class); STUB_CONTRACTS.put(FileInputFormat.class, GenericDataSourceBase.class); STUB_CONTRACTS.put(FileOutputFormat.class, GenericDataSinkBase.class); STUB_CONTRACTS.put(InputFormat.class, GenericDataSourceBase.class); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java index 38688b6aa3088..30091ab53df47 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/util/OperatorUtilTest.java @@ -21,12 +21,12 @@ import static org.junit.Assert.assertEquals; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.common.functions.FlatJoinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.GenericCollectorMap; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.io.DelimitedInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; @@ -115,13 +115,13 @@ public void getContractClassShouldReturnSourceForInputFormat() { assertEquals(GenericDataSourceBase.class, result); } - static abstract class CoGrouper implements CoGroupable {} + static abstract class CoGrouper implements CoGroupFunction {} - static abstract class Crosser implements Crossable {} + static abstract class Crosser implements CrossFunction {} static abstract class Mapper implements GenericCollectorMap {} - static abstract class Matcher implements FlatJoinable {} + static abstract class Matcher implements FlatJoinFunction {} - static abstract class Reducer implements GroupReducible {} + static abstract class Reducer implements GroupReduceFunction {} } diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java index 4c22db1a29a0c..8767aca8e4495 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java @@ -21,8 +21,9 @@ import java.io.Serializable; import java.util.Collection; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -197,7 +198,7 @@ public String toString() { // ************************************************************************* /** Converts a Tuple2 into a Point. */ - public static final class TuplePointConverter extends MapFunction, Point> { + public static final class TuplePointConverter implements MapFunction, Point> { @Override public Point map(Tuple2 t) throws Exception { @@ -206,7 +207,7 @@ public Point map(Tuple2 t) throws Exception { } /** Converts a Tuple3 into a Centroid. */ - public static final class TupleCentroidConverter extends MapFunction, Centroid> { + public static final class TupleCentroidConverter implements MapFunction, Centroid> { @Override public Centroid map(Tuple3 t) throws Exception { @@ -215,7 +216,7 @@ public Centroid map(Tuple3 t) throws Exception { } /** Determines the closest cluster center for a data point. */ - public static final class SelectNearestCenter extends MapFunction> { + public static final class SelectNearestCenter extends RichMapFunction> { private Collection centroids; /** Reads the centroid values from a broadcast variable into a collection. */ @@ -248,7 +249,7 @@ public Tuple2 map(Point p) throws Exception { } /** Appends a count variable to the tuple. */ - public static final class CountAppender extends MapFunction, Tuple3> { + public static final class CountAppender implements MapFunction, Tuple3> { @Override public Tuple3 map(Tuple2 t) { @@ -257,7 +258,7 @@ public Tuple3 map(Tuple2 t) { } /** Sums and counts point coordinates. */ - public static final class CentroidAccumulator extends ReduceFunction> { + public static final class CentroidAccumulator implements ReduceFunction> { @Override public Tuple3 reduce(Tuple3 val1, Tuple3 val2) { @@ -266,7 +267,7 @@ public Tuple3 reduce(Tuple3 val1, Tu } /** Computes new centroid from coordinate sum and count of points. */ - public static final class CentroidAverager extends MapFunction, Centroid> { + public static final class CentroidAverager implements MapFunction, Centroid> { @Override public Centroid map(Tuple3 value) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java index bf96474a81f8d..0d38e063b4c37 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java @@ -20,14 +20,14 @@ package org.apache.flink.example.java.graph; import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.FlatJoinFunction; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; -import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; @@ -128,7 +128,7 @@ public static void main(String... args) throws Exception { * Function that turns a value into a 2-tuple where both fields are that value. */ @ConstantFields("0 -> 0,1") - public static final class DuplicateValue extends MapFunction> { + public static final class DuplicateValue implements MapFunction> { @Override public Tuple2 map(T vertex) { @@ -139,7 +139,7 @@ public Tuple2 map(T vertex) { /** * Undirected edges by emitting for each input edge the input edges itself and an inverted version. */ - public static final class UndirectEdge extends FlatMapFunction, Tuple2> { + public static final class UndirectEdge implements FlatMapFunction, Tuple2> { Tuple2 invertedEdge = new Tuple2(); @Override @@ -158,7 +158,7 @@ public void flatMap(Tuple2 edge, Collector> out) */ @ConstantFieldsFirst("1 -> 0") @ConstantFieldsSecond("1 -> 1") - public static final class NeighborWithComponentIDJoin extends JoinFunction, Tuple2, Tuple2> { + public static final class NeighborWithComponentIDJoin implements JoinFunction, Tuple2, Tuple2> { @Override public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) { @@ -169,7 +169,7 @@ public Tuple2 join(Tuple2 vertexWithComponent, Tuple2, Tuple2, Tuple2> { + public static final class ComponentIdFilter implements FlatJoinFunction, Tuple2, Tuple2> { @Override public void join(Tuple2 candidate, Tuple2 old, Collector> out) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java index fba18fcecbcc5..2d794bd5b4540 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java @@ -22,10 +22,10 @@ import java.util.Iterator; import java.util.List; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; import org.apache.flink.api.java.DataSet; @@ -119,7 +119,7 @@ public static void main(String[] args) throws Exception { // ************************************************************************* /** Converts a Tuple2 into an Edge */ - public static class TupleEdgeConverter extends MapFunction, Edge> { + public static class TupleEdgeConverter implements MapFunction, Edge> { private final Edge outEdge = new Edge(); @Override @@ -130,7 +130,7 @@ public Edge map(Tuple2 t) throws Exception { } /** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */ - private static class EdgeByIdProjector extends MapFunction { + private static class EdgeByIdProjector implements MapFunction { @Override public Edge map(Edge inEdge) throws Exception { @@ -149,7 +149,7 @@ public Edge map(Edge inEdge) throws Exception { * The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. * Assumes that input edges share the first vertex and are in ascending order of the second vertex. */ - private static class TriadBuilder extends GroupReduceFunction { + private static class TriadBuilder implements GroupReduceFunction { private final List vertices = new ArrayList(); private final Triad outTriad = new Triad(); @@ -180,7 +180,7 @@ public void reduce(Iterator edges, Collector out) throws Exception } /** Filters triads (three vertices connected by two edges) without a closing third edge. */ - private static class TriadFilter extends JoinFunction { + private static class TriadFilter implements JoinFunction { @Override public Triad join(Triad triad, Edge edge) throws Exception { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java index 265ce752f7423..c0ea26a3e2229 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java @@ -22,12 +22,12 @@ import java.util.Iterator; import java.util.List; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; import org.apache.flink.api.java.DataSet; @@ -134,7 +134,7 @@ public static void main(String[] args) throws Exception { // ************************************************************************* /** Converts a Tuple2 into an Edge */ - public static class TupleEdgeConverter extends MapFunction, Edge> { + public static class TupleEdgeConverter implements MapFunction, Edge> { private final Edge outEdge = new Edge(); @Override @@ -145,7 +145,7 @@ public Edge map(Tuple2 t) throws Exception { } /** Emits for an edge the original edge and its switched version. */ - private static class EdgeDuplicator extends FlatMapFunction { + private static class EdgeDuplicator implements FlatMapFunction { @Override public void flatMap(Edge edge, Collector out) throws Exception { @@ -160,7 +160,7 @@ public void flatMap(Edge edge, Collector out) throws Exception { * Emits one edge for each input edge with a degree annotation for the shared vertex. * For each emitted edge, the first vertex is the vertex with the smaller id. */ - private static class DegreeCounter extends GroupReduceFunction { + private static class DegreeCounter implements GroupReduceFunction { final ArrayList otherVertices = new ArrayList(); final EdgeWithDegrees outputEdge = new EdgeWithDegrees(); @@ -208,7 +208,7 @@ public void reduce(Iterator edges, Collector out) throws * Builds an edge with degree annotation from two edges that have the same vertices and only one * degree annotation. */ - private static class DegreeJoiner extends ReduceFunction { + private static class DegreeJoiner implements ReduceFunction { private final EdgeWithDegrees outEdge = new EdgeWithDegrees(); @Override @@ -228,7 +228,7 @@ public EdgeWithDegrees reduce(EdgeWithDegrees edge1, EdgeWithDegrees edge2) thro } /** Projects an edge (pair of vertices) such that the first vertex is the vertex with the smaller degree. */ - private static class EdgeByDegreeProjector extends MapFunction { + private static class EdgeByDegreeProjector implements MapFunction { private final Edge outEdge = new Edge(); @@ -249,7 +249,7 @@ public Edge map(EdgeWithDegrees inEdge) throws Exception { } /** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */ - private static class EdgeByIdProjector extends MapFunction { + private static class EdgeByIdProjector implements MapFunction { @Override public Edge map(Edge inEdge) throws Exception { @@ -268,7 +268,7 @@ public Edge map(Edge inEdge) throws Exception { * The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. * Assumes that input edges share the first vertex and are in ascending order of the second vertex. */ - private static class TriadBuilder extends GroupReduceFunction { + private static class TriadBuilder implements GroupReduceFunction { private final List vertices = new ArrayList(); private final Triad outTriad = new Triad(); @@ -300,7 +300,7 @@ public void reduce(Iterator edges, Collector out) throws Exception } /** Filters triads (three vertices connected by two edges) without a closing third edge. */ - private static class TriadFilter extends JoinFunction { + private static class TriadFilter implements JoinFunction { @Override public Triad join(Triad triad, Edge edge) throws Exception { @@ -332,7 +332,7 @@ private static boolean parseParameters(String[] args) { System.out.println("Executing Enum Triangles Opt example with built-in default data."); System.out.println(" Provide parameters to read input data from files."); System.out.println(" See the documentation for the correct format of input files."); - System.out.println(" Usage: EnumTriangleBasic "); + System.out.println(" Usage: EnumTriangleOpt "); } return true; } diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java index 18eba5d3deca5..ba9754fa0b51c 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java @@ -23,10 +23,10 @@ import java.util.ArrayList; import java.util.Iterator; -import org.apache.flink.api.java.functions.FilterFunction; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; @@ -139,7 +139,7 @@ public static void main(String[] args) throws Exception { /** * A map function that assigns an initial rank to all pages. */ - public static final class RankAssigner extends MapFunction, Tuple2> { + public static final class RankAssigner implements MapFunction, Tuple2> { Tuple2 outPageWithRank; public RankAssigner(double rank) { @@ -158,7 +158,7 @@ public Tuple2 map(Tuple1 page) { * originate. Run as a pre-processing step. */ @ConstantFields("0") - public static final class BuildOutgoingEdgeList extends GroupReduceFunction, Tuple2> { + public static final class BuildOutgoingEdgeList implements GroupReduceFunction, Tuple2> { private final ArrayList neighbors = new ArrayList(); @@ -179,7 +179,7 @@ public void reduce(Iterator> values, Collector, Tuple2>, Tuple2> { + public static final class JoinVertexWithEdgesMatch implements FlatMapFunction, Tuple2>, Tuple2> { @Override public void flatMap(Tuple2, Tuple2> value, Collector> out){ @@ -197,7 +197,7 @@ public void flatMap(Tuple2, Tuple2> value, Co * The function that applies the page rank dampening formula */ @ConstantFields("0") - public static final class Dampener extends MapFunction, Tuple2> { + public static final class Dampener implements MapFunction, Tuple2> { private final double dampening; private final double randomJump; @@ -217,7 +217,7 @@ public Tuple2 map(Tuple2 value) { /** * Filter that filters vertices where the rank difference is below a threshold. */ - public static final class EpsilonFilter extends FilterFunction, Tuple2>> { + public static final class EpsilonFilter implements FilterFunction, Tuple2>> { @Override public boolean filter(Tuple2, Tuple2> value) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java index d8d8b62f75c62..22054da055966 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java @@ -19,6 +19,8 @@ package org.apache.flink.example.java.graph; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; @@ -26,8 +28,6 @@ import java.util.Iterator; import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.example.java.graph.util.ConnectedComponentsData; import org.apache.flink.util.Collector; diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java index 1d687f32e5bd5..0868732b8f71c 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java @@ -22,8 +22,9 @@ import java.io.Serializable; import java.util.Collection; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.example.java.ml.util.LinearRegressionData; @@ -183,7 +184,7 @@ public Params div(Integer a){ // ************************************************************************* /** Converts a Tuple2 into a Data. */ - public static final class TupleDataConverter extends MapFunction, Data> { + public static final class TupleDataConverter implements MapFunction, Data> { @Override public Data map(Tuple2 t) throws Exception { @@ -192,7 +193,7 @@ public Data map(Tuple2 t) throws Exception { } /** Converts a Tuple2 into a Params. */ - public static final class TupleParamsConverter extends MapFunction,Params> { + public static final class TupleParamsConverter implements MapFunction,Params> { @Override public Params map(Tuple2 t)throws Exception { @@ -203,7 +204,7 @@ public Params map(Tuple2 t)throws Exception { /** * Compute a single BGD type update for every parameters. */ - public static class SubUpdate extends MapFunction>{ + public static class SubUpdate extends RichMapFunction> { private Collection parameters; @@ -234,7 +235,7 @@ public Tuple2 map(Data in) throws Exception { /** * Accumulator all the update. * */ - public static class UpdateAccumulator extends ReduceFunction> { + public static class UpdateAccumulator implements ReduceFunction> { @Override public Tuple2 reduce(Tuple2 val1, Tuple2 val2) { @@ -250,7 +251,7 @@ public Tuple2 reduce(Tuple2 val1, Tuple2,Params>{ + public static class Update implements MapFunction,Params> { @Override public Params map(Tuple2 arg0) throws Exception { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java index 61b90dddf7886..4bced17c0f614 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java @@ -26,7 +26,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FilterFunction; +import org.apache.flink.api.java.functions.RichFilterFunction; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -149,7 +149,7 @@ private static Collection> getExampleInputTuples( * In doing so, it also counts the number of empty fields per attribute with an accumulator (registered under * {@link EmptyFieldsCountAccumulator#EMPTY_FIELD_ACCUMULATOR}). */ - public static final class EmptyFieldFilter extends FilterFunction { + public static final class EmptyFieldFilter extends RichFilterFunction { // create a new accumulator in each filter function instance // accumulators can be merged later on diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java index 48cdedc8a32e0..08a261c479f28 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java @@ -18,8 +18,8 @@ package org.apache.flink.example.java.relational; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java index ef03e6f97be33..1ff6583a84572 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java @@ -18,9 +18,9 @@ package org.apache.flink.example.java.relational; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FilterFunction; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple4; diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java index 52109eab453e2..4544fd440e1f0 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java @@ -25,9 +25,9 @@ import java.util.Calendar; import java.util.Date; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FilterFunction; -import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple4; diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java index 2649d24fe0867..3033c0df1620d 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java @@ -21,8 +21,8 @@ import java.util.Iterator; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.FilterFunction; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; @@ -159,7 +159,7 @@ public static void main(String[] args) throws Exception { * MapFunction that filters for documents that contain a certain set of * keywords. */ - public static class FilterDocByKeyWords extends FilterFunction> { + public static class FilterDocByKeyWords implements FilterFunction> { private static final String[] KEYWORDS = { " editors ", " oscillations " }; @@ -187,7 +187,7 @@ public boolean filter(Tuple2 value) throws Exception { /** * MapFunction that filters for records where the rank exceeds a certain threshold. */ - public static class FilterByRank extends FilterFunction> { + public static class FilterByRank implements FilterFunction> { private static final int RANKFILTER = 40; @@ -210,7 +210,7 @@ public boolean filter(Tuple3 value) throws Exception { * MapFunction that filters for records of the visits relation where the year * (from the date string) is equal to a certain value. */ - public static class FilterVisitsByDate extends FilterFunction> { + public static class FilterVisitsByDate implements FilterFunction> { private static final int YEARFILTER = 2007; @@ -237,7 +237,7 @@ public boolean filter(Tuple2 value) throws Exception { * If the first input does not provide any pairs, all pairs of the second input are emitted. * Otherwise, no pair is emitted. */ - public static class AntiJoinVisits extends CoGroupFunction, Tuple1, Tuple3> { + public static class AntiJoinVisits implements CoGroupFunction, Tuple1, Tuple3> { /** * If the visit iterator is empty, all pairs of the rank iterator are emitted. diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java index a18abcb673b13..3e95ccd0264f9 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java @@ -18,7 +18,7 @@ package org.apache.flink.example.java.wordcount; -import org.apache.flink.api.java.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; import org.apache.flink.api.java.DataSet; @@ -91,7 +91,7 @@ public static void main(String[] args) throws Exception { * FlatMapFunction. The function takes a line (String) and splits it into * multiple pairs in the form of "(word,1)" (Tuple2). */ - public static final class Tokenizer extends FlatMapFunction> { + public static final class Tokenizer implements FlatMapFunction> { @Override public void flatMap(String value, Collector> out) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCountPOJO.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCountPOJO.java index 32a8997d980dc..04810a15ac3ab 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCountPOJO.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCountPOJO.java @@ -18,8 +18,8 @@ package org.apache.flink.example.java.wordcount; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.util.Collector; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -111,7 +111,7 @@ public String toString() { * FlatMapFunction. The function takes a line (String) and splits it into * multiple WC POJOs as "(word, 1)". */ - public static final class Tokenizer extends FlatMapFunction { + public static final class Tokenizer implements FlatMapFunction { @Override public void flatMap(String value, Collector out) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index b83c695e1673d..e7199f9240cd1 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -19,22 +19,16 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.Validate; -import org.apache.flink.api.common.functions.Filterable; -import org.apache.flink.api.common.functions.FlatMappable; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.Mappable; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.FilterFunction; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.io.CsvOutputFormat; import org.apache.flink.api.java.io.PrintingOutputFormat; @@ -71,8 +65,8 @@ * A DataSet represents a collection of elements of the same type.
* A DataSet can be transformed into another DataSet by applying a transformation as for example *
    - *
  • {@link DataSet#map(MapFunction)},
  • - *
  • {@link DataSet#reduce(ReduceFunction)},
  • + *
  • {@link DataSet#map(org.apache.flink.api.java.functions.RichMapFunction)},
  • + *
  • {@link DataSet#reduce(org.apache.flink.api.java.functions.RichReduceFunction)},
  • *
  • {@link DataSet#join(DataSet)}, or
  • *
  • {@link DataSet#coGroup(DataSet)}.
  • *
@@ -127,17 +121,17 @@ public TypeInformation getType() { /** * Applies a Map transformation on a {@link DataSet}.
- * The transformation calls a {@link MapFunction} for each element of the DataSet. + * The transformation calls a {@link org.apache.flink.api.java.functions.RichMapFunction} for each element of the DataSet. * Each MapFunction call returns exactly one element. * * @param mapper The MapFunction that is called for each element of the DataSet. * @return A MapOperator that represents the transformed DataSet. * - * @see MapFunction + * @see org.apache.flink.api.java.functions.RichMapFunction * @see MapOperator * @see DataSet */ - public MapOperator map(Mappable mapper) { + public MapOperator map(MapFunction mapper) { if (mapper == null) { throw new NullPointerException("Map function must not be null."); } @@ -149,17 +143,17 @@ public MapOperator map(Mappable mapper) { /** * Applies a FlatMap transformation on a {@link DataSet}.
- * The transformation calls a {@link FlatMapFunction} for each element of the DataSet. + * The transformation calls a {@link org.apache.flink.api.java.functions.RichFlatMapFunction} for each element of the DataSet. * Each FlatMapFunction call can return any number of elements including none. * * @param flatMapper The FlatMapFunction that is called for each element of the DataSet. * @return A FlatMapOperator that represents the transformed DataSet. * - * @see FlatMapFunction + * @see org.apache.flink.api.java.functions.RichFlatMapFunction * @see FlatMapOperator * @see DataSet */ - public FlatMapOperator flatMap(FlatMappable flatMapper) { + public FlatMapOperator flatMap(FlatMapFunction flatMapper) { if (flatMapper == null) { throw new NullPointerException("FlatMap function must not be null."); } @@ -171,18 +165,18 @@ public FlatMapOperator flatMap(FlatMappable flatMapper) { /** * Applies a Filter transformation on a {@link DataSet}.
- * The transformation calls a {@link FilterFunction} for each element of the DataSet + * The transformation calls a {@link org.apache.flink.api.java.functions.RichFilterFunction} for each element of the DataSet * and retains only those element for which the function returns true. Elements for * which the function returns false are filtered. * * @param filter The FilterFunction that is called for each element of the DataSet. * @return A FilterOperator that represents the filtered DataSet. * - * @see FilterFunction + * @see org.apache.flink.api.java.functions.RichFilterFunction * @see FilterOperator * @see DataSet */ - public FilterOperator filter(Filterable filter) { + public FilterOperator filter(FilterFunction filter) { if (filter == null) { throw new NullPointerException("Filter function must not be null."); } @@ -274,18 +268,18 @@ public AggregateOperator min (int field) { /** * Applies a Reduce transformation on a non-grouped {@link DataSet}.
- * The transformation consecutively calls a {@link ReduceFunction} + * The transformation consecutively calls a {@link org.apache.flink.api.java.functions.RichReduceFunction} * until only a single element remains which is the result of the transformation. * A ReduceFunction combines two elements into one new element of the same type. * * @param reducer The ReduceFunction that is applied on the DataSet. * @return A ReduceOperator that represents the reduced DataSet. * - * @see ReduceFunction + * @see org.apache.flink.api.java.functions.RichReduceFunction * @see ReduceOperator * @see DataSet */ - public ReduceOperator reduce(Reducible reducer) { + public ReduceOperator reduce(ReduceFunction reducer) { if (reducer == null) { throw new NullPointerException("Reduce function must not be null."); } @@ -294,18 +288,18 @@ public ReduceOperator reduce(Reducible reducer) { /** * Applies a GroupReduce transformation on a non-grouped {@link DataSet}.
- * The transformation calls a {@link GroupReduceFunction} once with the full DataSet. + * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} once with the full DataSet. * The GroupReduceFunction can iterate over all elements of the DataSet and emit any * number of output elements including none. * * @param reducer The GroupReduceFunction that is applied on the DataSet. * @return A GroupReduceOperator that represents the reduced DataSet. * - * @see GroupReduceFunction + * @see org.apache.flink.api.java.functions.RichGroupReduceFunction * @see org.apache.flink.api.java.operators.GroupReduceOperator * @see DataSet */ - public GroupReduceOperator reduceGroup(GroupReducible reducer) { + public GroupReduceOperator reduceGroup(GroupReduceFunction reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } @@ -372,8 +366,8 @@ public DistinctOperator distinct() { *
    *
  • {@link UnsortedGrouping#sortGroup(int, org.apache.flink.api.common.operators.Order)} to get a {@link SortedGrouping}. *
  • {@link UnsortedGrouping#aggregate(Aggregations, int)} to apply an Aggregate transformation. - *
  • {@link UnsortedGrouping#reduce(ReduceFunction)} to apply a Reduce transformation. - *
  • {@link UnsortedGrouping#reduceGroup(GroupReduceFunction)} to apply a GroupReduce transformation. + *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.RichReduceFunction)} to apply a Reduce transformation. + *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)} to apply a GroupReduce transformation. *
* * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is grouped. @@ -401,8 +395,8 @@ public > UnsortedGrouping groupBy(KeySelector k *
    *
  • {@link UnsortedGrouping#sortGroup(int, org.apache.flink.api.common.operators.Order)} to get a {@link SortedGrouping}. *
  • {@link UnsortedGrouping#aggregate(Aggregations, int)} to apply an Aggregate transformation. - *
  • {@link UnsortedGrouping#reduce(ReduceFunction)} to apply a Reduce transformation. - *
  • {@link UnsortedGrouping#reduceGroup(GroupReduceFunction)} to apply a GroupReduce transformation. + *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.RichReduceFunction)} to apply a Reduce transformation. + *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)} to apply a GroupReduce transformation. *
* * @param fields One or more field positions on which the DataSet will be grouped. @@ -430,8 +424,8 @@ public UnsortedGrouping groupBy(int... fields) { *
    *
  • {@link UnsortedGrouping#sortGroup(int, org.apache.flink.api.common.operators.Order)} to get a {@link SortedGrouping}. *
  • {@link UnsortedGrouping#aggregate(Aggregations, int)} to apply an Aggregate transformation. - *
  • {@link UnsortedGrouping#reduce(ReduceFunction)} to apply a Reduce transformation. - *
  • {@link UnsortedGrouping#reduceGroup(GroupReduceFunction)} to apply a GroupReduce transformation. + *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.RichReduceFunction)} to apply a Reduce transformation. + *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)} to apply a GroupReduce transformation. *
* * @param fields One or more field expressions on which the DataSet will be grouped. @@ -527,7 +521,7 @@ public JoinOperatorSets joinWithHuge(DataSet other) { * Initiates a CoGroup transformation.
* A CoGroup transformation combines the elements of * two {@link DataSet DataSets} into one DataSet. It groups each DataSet individually on a key and - * gives groups of both DataSets with equal keys together into a {@link CoGroupFunction}. + * gives groups of both DataSets with equal keys together into a {@link org.apache.flink.api.java.functions.RichCoGroupFunction}. * If a DataSet has a group with no matching key in the other DataSet, the CoGroupFunction * is called with an empty group for the non-existing group.
* The CoGroupFunction can iterate over the elements of both groups and return any number diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index 3c3b9a0aa6843..ebd1422cda02a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -112,7 +112,7 @@ protected ExecutionEnvironment() { * individually override this value to use a specific degree of parallelism via * {@link Operator#setParallelism(int)}. Other operations may need to run with a different * degree of parallelism - for example calling - * {@link DataSet#reduce(org.apache.flink.api.java.functions.ReduceFunction)} over the entire + * {@link DataSet#reduce(org.apache.flink.api.java.functions.RichReduceFunction)} over the entire * set will insert eventually an operation that runs non-parallel (degree of parallelism of one). * * @return The degree of parallelism used by operations, unless they override that value. This method diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java index 5cc8c128bfb27..012ab5737ae18 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java @@ -23,7 +23,7 @@ import org.apache.flink.util.Collector; /** - * A variant of the {@link FlatMapFunction} that returns elements through an iterator, rather then + * A variant of the {@link RichFlatMapFunction} that returns elements through an iterator, rather then * through a collector. In all other respects, it behaves exactly like the FlatMapFunction. *

* The function needs to be serializable, as defined in {@link java.io.Serializable}. @@ -31,7 +31,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class FlatMapIterator extends FlatMapFunction { +public abstract class FlatMapIterator extends RichFlatMapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java index 8e2f2b2c54fd0..b94840f4baaba 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java @@ -64,15 +64,15 @@ * *

* Be aware that some annotations should only be used for functions with as single input - * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs - * ({@link CrossFunction}, {@link FlatJoinFunction}, {@link CoGroupFunction}). + * ({@link RichMapFunction}, {@link RichReduceFunction}) and some only for stubs with two inputs + * ({@link RichCrossFunction}, {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}). */ public class FunctionAnnotation { /** * This annotation declares that a function leaves certain fields of its input values unmodified and * only "forwards" or "copies" them to the return value. The annotation is applicable to unary - * functions, like for example {@link MapFunction}, {@link ReduceFunction}, or {@link FlatMapFunction}. + * functions, like for example {@link RichMapFunction}, {@link RichReduceFunction}, or {@link RichFlatMapFunction}. *

* The following example illustrates a function that keeps the tuple's field zero constant: *

@@ -103,7 +103,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function leaves certain fields of its first input values unmodified and * only "forwards" or "copies" them to the return value. The annotation is applicable to binary - * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -135,7 +135,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function leaves certain fields of its second input values unmodified and * only "forwards" or "copies" them to the return value. The annotation is applicable to binary - * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -167,7 +167,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function changes certain fields of its input values, while leaving all * others unmodified and in place in the return value. The annotation is applicable to unary - * functions, like for example {@link MapFunction}, {@link ReduceFunction}, or {@link FlatMapFunction}. + * functions, like for example {@link RichMapFunction}, {@link RichReduceFunction}, or {@link RichFlatMapFunction}. *

* The following example illustrates that at the example of a Map function: * @@ -201,7 +201,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function changes certain fields of its first input value, while leaving all * others unmodified and in place in the return value. The annotation is applicable to binary - * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: @@ -238,7 +238,7 @@ public class FunctionAnnotation { /** * This annotation declares that a function changes certain fields of its second input value, while leaving all * others unmodified and in place in the return value. The annotation is applicable to binary - * functions, like for example {@link FlatJoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}. *

* The following example illustrates a join function that copies fields from the first and second input to the * return value: diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java index 6cb397b93d4cd..b363606c02faf 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java @@ -23,7 +23,7 @@ import org.apache.flink.util.Collector; -public abstract class GroupReduceIterator extends GroupReduceFunction { +public abstract class GroupReduceIterator extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java index 3f49314f84c03..8aaaf86ac76c2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java @@ -21,7 +21,7 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.util.Collector; /** @@ -50,7 +50,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class RichCoGroupFunction extends AbstractRichFunction implements CoGroupable { +public abstract class RichCoGroupFunction extends AbstractRichFunction implements CoGroupFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java index 6a64834086385..a4e1248dd087a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; /** @@ -45,7 +45,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class RichCrossFunction extends AbstractRichFunction implements Crossable { +public abstract class RichCrossFunction extends AbstractRichFunction implements CrossFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java index 78d126c8d6f70..e3baa746014d8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Filterable; +import org.apache.flink.api.common.functions.FilterFunction; /** * The abstract base class for Filter functions. A filter function take elements and evaluates a @@ -36,7 +36,7 @@ * * @param The type of the filtered elements. */ -public abstract class RichFilterFunction extends AbstractRichFunction implements Filterable { +public abstract class RichFilterFunction extends AbstractRichFunction implements FilterFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java index e369d9ad564ab..8c326c696d743 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java @@ -20,12 +20,12 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.util.Collector; import java.util.Iterator; -public abstract class RichFlatCombineFunction extends AbstractRichFunction implements FlatCombinable { +public abstract class RichFlatCombineFunction extends AbstractRichFunction implements FlatCombineFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java index 1dab5bce48316..15b453918e921 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.util.Collector; /** @@ -47,7 +47,7 @@ * the result of the operation is a sequence of Tuple2, where the elements in the tuple are those that * the JoinFunction would have been invoked with. *

- * Note: You can use a {@link CoGroupFunction} to perform an outer join. + * Note: You can use a {@link RichCoGroupFunction} to perform an outer join. *

* All functions need to be serializable, as defined in {@link java.io.Serializable}. * @@ -55,7 +55,7 @@ * @param The type of the elements in the second input. * @param The type of the result elements. */ -public abstract class RichFlatJoinFunction extends AbstractRichFunction implements FlatJoinable { +public abstract class RichFlatJoinFunction extends AbstractRichFunction implements FlatJoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java index 02398957160ce..2293b5ebaf211 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; /** @@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class RichFlatMapFunction extends AbstractRichFunction implements FlatMappable { +public abstract class RichFlatMapFunction extends AbstractRichFunction implements FlatMapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java index 2e37780a5245b..eb75f53217fc6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java @@ -25,8 +25,8 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; /** @@ -51,7 +51,7 @@ * @param Type of the elements that this function processes. * @param The type of the elements returned by the user-defined function. */ -public abstract class RichGroupReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { +public abstract class RichGroupReduceFunction extends AbstractRichFunction implements GroupReduceFunction, FlatCombineFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java index 9c0608720ed89..7eaf44c6b89b0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Joinable; +import org.apache.flink.api.common.functions.JoinFunction; -public abstract class RichJoinFunction extends AbstractRichFunction implements Joinable { +public abstract class RichJoinFunction extends AbstractRichFunction implements JoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java index 87485b08dfb8b..54de7d4681c8e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java @@ -19,14 +19,14 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; /** * The abstract base class for Map functions. Map functions take elements and transform them, * element wise. A Map function always produces a single result element for each input element. * Typical applications are parsing elements, converting data types, or projecting out fields. * Operations that produce multiple result elements from a single input element can be implemented - * using the {@link FlatMapFunction}. + * using the {@link RichFlatMapFunction}. *

* The basic syntax for using a MapFunction is as follows: *

@@ -40,7 +40,7 @@ * @param Type of the input elements. * @param Type of the returned elements. */ -public abstract class RichMapFunction extends AbstractRichFunction implements Mappable { +public abstract class RichMapFunction extends AbstractRichFunction implements MapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java index 50580cf431e81..35cb392eb68e8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; /** * The abstract base class for Reduce functions. Reduce functions combine groups of elements to @@ -28,8 +28,8 @@ * individually. *

* For a reduce functions that work on an entire group at the same time (such as the - * MapReduce/Hadoop-style reduce), see {@link GroupReduceFunction}, called via - * {@link org.apache.flink.api.java.DataSet#reduceGroup(GroupReduceFunction)}. In the general case, + * MapReduce/Hadoop-style reduce), see {@link RichGroupReduceFunction}, called via + * {@link org.apache.flink.api.java.DataSet#reduceGroup(RichGroupReduceFunction)}. In the general case, * ReduceFunctions are considered faster, because they allow the system to use hash-based * execution strategies. *

@@ -44,7 +44,7 @@ * * @param Type of the elements that this function processes. */ -public abstract class RichReduceFunction extends AbstractRichFunction implements Reducible { +public abstract class RichReduceFunction extends AbstractRichFunction implements ReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index f0f7ca40a9131..80a5fa04d57ce 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -24,7 +24,7 @@ import org.apache.commons.lang3.Validate; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -32,8 +32,8 @@ import org.apache.flink.api.java.aggregation.AggregationFunction; import org.apache.flink.api.java.aggregation.AggregationFunctionFactory; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; @@ -151,7 +151,7 @@ public AggregateOperator andMax (int field) { @SuppressWarnings("unchecked") @Override - protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase> translateToDataFlow(Operator input) { // sanity check if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) { @@ -174,7 +174,7 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase function = new AggregatingUdf(aggFunctions, fields); + RichGroupReduceFunction function = new AggregatingUdf(aggFunctions, fields); String name = getName() != null ? getName() : genName.toString(); @@ -183,8 +183,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); po.setCombinable(true); @@ -200,8 +200,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(true); @@ -245,7 +245,7 @@ else if (this.grouping.getKeys() instanceof Keys.SelectorFunctionKeys) { // -------------------------------------------------------------------------------------------- @Combinable - public static final class AggregatingUdf extends GroupReduceFunction { + public static final class AggregatingUdf extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; private final int[] fieldPositions; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index b9847e85bdb09..89c33342b3fe7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -21,8 +21,8 @@ import java.security.InvalidParameterException; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Operator; @@ -31,7 +31,6 @@ import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; import org.apache.flink.api.java.operators.Keys.FieldPositionKeys; @@ -55,7 +54,7 @@ */ public class CoGroupOperator extends TwoInputUdfOperator> { - private final CoGroupable function; + private final CoGroupFunction function; private final Keys keys1; private final Keys keys2; @@ -63,7 +62,7 @@ public class CoGroupOperator extends TwoInputUdfOperator input1, DataSet input2, Keys keys1, Keys keys2, - CoGroupable function, + CoGroupFunction function, TypeInformation returnType) { super(input1, input2, returnType); @@ -125,8 +124,8 @@ else if ((keys1 instanceof Keys.FieldPositionKeys int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - CoGroupOperatorBase> po = - new CoGroupOperatorBase>( + CoGroupOperatorBase> po = + new CoGroupOperatorBase>( function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); @@ -186,7 +185,7 @@ else if (keys1 instanceof Keys.SelectorFunctionKeys private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroup( Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupable function, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -201,10 +200,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); cogroup.setFirstInput(keyMapper1); @@ -221,7 +220,7 @@ private static PlanUnwrappingCoGroupOperator tr private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupRight( int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupable function, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -238,10 +237,10 @@ private static PlanUnwrappingCoGroupOperator tr final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -259,7 +258,7 @@ private static PlanUnwrappingCoGroupOperator tr private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupLeft( Keys.SelectorFunctionKeys rawKeys1, int[] logicalKeyPositions2, - CoGroupable function, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -276,10 +275,10 @@ private static PlanUnwrappingCoGroupOperator tr final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -409,7 +408,7 @@ private CoGroupOperatorSetsPredicate(Keys keys1) { * @param field0 The first index of the Tuple fields of the second co-grouped DataSets that should be used as key * @param fields The indexes of the Tuple fields of the second co-grouped DataSet that should be used as keys. * @return An incomplete CoGroup transformation. - * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(CoGroupFunction)} to finalize the CoGroup transformation. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.java.functions.RichCoGroupFunction)} to finalize the CoGroup transformation. */ public CoGroupOperatorWithoutFunction equalTo(int field0, int... fields) { int[] actualFields = new int[fields.length + 1]; @@ -425,7 +424,7 @@ public CoGroupOperatorWithoutFunction equalTo(int field0, int... fields) { * @param field0 The first field of the second co-grouped DataSets that should be used as key * @param fields The fields of the first co-grouped DataSets that should be used as keys. * @return An incomplete CoGroup transformation. - * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(CoGroupFunction)} to finalize the CoGroup transformation. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.java.functions.RichCoGroupFunction)} to finalize the CoGroup transformation. */ public CoGroupOperatorWithoutFunction equalTo(String field0, String... fields) { String[] actualFields = new String[fields.length + 1]; @@ -441,7 +440,7 @@ public CoGroupOperatorWithoutFunction equalTo(String field0, String... fields) { * * @param keyExtractor The KeySelector function which extracts the key values from the second DataSet on which it is grouped. * @return An incomplete CoGroup transformation. - * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(CoGroupFunction)} to finalize the CoGroup transformation. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.java.functions.RichCoGroupFunction)} to finalize the CoGroup transformation. */ public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtractor) { return createCoGroupOperator(new Keys.SelectorFunctionKeys(keyExtractor, input2.getType())); @@ -449,8 +448,8 @@ public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtracto /** * Intermediate step of a CoGroup transformation.
- * To continue the CoGroup transformation, provide a {@link CoGroupFunction} by calling - * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(CoGroupFunction)}. + * To continue the CoGroup transformation, provide a {@link org.apache.flink.api.java.functions.RichCoGroupFunction} by calling + * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.java.functions.RichCoGroupFunction)}. * */ private CoGroupOperatorWithoutFunction createCoGroupOperator(Keys keys2) { @@ -503,16 +502,16 @@ private CoGroupOperatorWithoutFunction(Keys keys2) { } /** - * Finalizes a CoGroup transformation by applying a {@link CoGroupFunction} to groups of elements with identical keys.
+ * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.java.functions.RichCoGroupFunction} to groups of elements with identical keys.
* Each CoGroupFunction call returns an arbitrary number of keys. * * @param function The CoGroupFunction that is called for all groups of elements with identical keys. * @return An CoGroupOperator that represents the co-grouped result DataSet. * - * @see CoGroupFunction + * @see org.apache.flink.api.java.functions.RichCoGroupFunction * @see DataSet */ - public CoGroupOperator with(CoGroupable function) { + public CoGroupOperator with(CoGroupFunction function) { if (function == null) { throw new NullPointerException("CoGroup function must not be null."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 93bb72ab69dee..d1e99d69bbf4f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -21,7 +21,7 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; @@ -49,10 +49,10 @@ */ public class CrossOperator extends TwoInputUdfOperator> { - private final Crossable function; + private final CrossFunction function; protected CrossOperator(DataSet input1, DataSet input2, - Crossable function, + CrossFunction function, TypeInformation returnType) { super(input1, input2, returnType); @@ -72,12 +72,12 @@ public void generateProjectionProperties(ProjectCrossFunction pcf) { } @Override - protected org.apache.flink.api.common.operators.base.CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { + protected org.apache.flink.api.common.operators.base.CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - CrossOperatorBase> po = - new CrossOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), name); + CrossOperatorBase> po = + new CrossOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), name); // set inputs po.setFirstInput(input1); po.setSecondInput(input2); @@ -109,7 +109,7 @@ public static final class DefaultCross extends CrossOperator input2; public DefaultCross(DataSet input1, DataSet input2) { - super(input1, input2, (Crossable>) new DefaultCrossFunction(), + super(input1, input2, (CrossFunction>) new DefaultCrossFunction(), new TupleTypeInfo>(input1.getType(), input2.getType())); if (input1 == null || input2 == null) { @@ -130,7 +130,7 @@ public DefaultCross(DataSet input1, DataSet input2) { * @see CrossFunction * @see DataSet */ - public CrossOperator with(Crossable function) { + public CrossOperator with(CrossFunction function) { if (function == null) { throw new NullPointerException("Cross function must not be null."); } @@ -224,7 +224,7 @@ public CrossOperator withConstantSetSecond(String... constantSetSec } } - public static final class ProjectCrossFunction implements Crossable { + public static final class ProjectCrossFunction implements CrossFunction { private static final long serialVersionUID = 1L; @@ -1402,7 +1402,7 @@ private TypeInformation[] extractFieldTypes(int[] fields, Class[] givenTyp // default join functions // -------------------------------------------------------------------------------------------- - public static final class DefaultCrossFunction implements Crossable> { + public static final class DefaultCrossFunction implements CrossFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index 40d6c2d546bb3..591551fbc3207 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -21,14 +21,14 @@ import java.util.Iterator; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.tuple.Tuple2; @@ -79,8 +79,8 @@ public DistinctOperator(DataSet input, Keys keys) { @Override protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase translateToDataFlow(Operator input) { - final GroupReduceFunction function = new DistinctFunction(); - final FlatCombinable combineFunction = new DistinctCombiner(); + final RichGroupReduceFunction function = new DistinctFunction(); + final FlatCombineFunction combineFunction = new DistinctCombiner(); String name = function.getClass().getName(); @@ -88,8 +88,8 @@ public DistinctOperator(DataSet input, Keys keys) { int[] logicalKeyPositions = keys.computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(true); po.setInput(input); @@ -118,7 +118,7 @@ else if (keys instanceof Keys.SelectorFunctionKeys) { // -------------------------------------------------------------------------------------------- private static PlanUnwrappingReduceGroupOperator translateSelectorFunctionDistinct( - Keys.SelectorFunctionKeys rawKeys, GroupReduceFunction function, FlatCombinable combineFunction, + Keys.SelectorFunctionKeys rawKeys, RichGroupReduceFunction function, FlatCombineFunction combineFunction, TypeInformation inputType, TypeInformation outputType, String name, Operator input, boolean combinable) { @@ -133,7 +133,7 @@ private static PlanUnwrappingReduceGroupOperator transl PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - MapOperatorBase, Mappable>> mapper = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, MapFunction>> mapper = new MapOperatorBase, MapFunction>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); @@ -144,7 +144,7 @@ private static PlanUnwrappingReduceGroupOperator transl return reducer; } - public static final class DistinctFunction extends GroupReduceFunction { + public static final class DistinctFunction extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; @@ -155,7 +155,7 @@ public void reduce(Iterator values, Collector out) } } - public static final class DistinctCombiner implements FlatCombinable { + public static final class DistinctCombiner implements FlatCombineFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java index 4733808e9e151..1c03ccd86d802 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.Filterable; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.operators.translation.PlanFilterOperator; @@ -33,10 +33,10 @@ */ public class FilterOperator extends SingleInputUdfOperator> { - protected final Filterable function; + protected final FilterFunction function; - public FilterOperator(DataSet input, Filterable function) { + public FilterOperator(DataSet input, FilterFunction function) { super(input, input.getType()); this.function = function; @@ -44,7 +44,7 @@ public FilterOperator(DataSet input, Filterable function) { } @Override - protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java index d3c3aa3925951..8e531d4df691e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; @@ -35,10 +35,10 @@ */ public class FlatMapOperator extends SingleInputUdfOperator> { - protected final FlatMappable function; + protected final FlatMapFunction function; - public FlatMapOperator(DataSet input, FlatMappable function) { + public FlatMapOperator(DataSet input, FlatMapFunction function) { super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType())); this.function = function; @@ -46,11 +46,11 @@ public FlatMapOperator(DataSet input, FlatMappable function) { } @Override - protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); + FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java index 1675280884cc7..e62b252a8963b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java @@ -18,16 +18,16 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.tuple.Tuple2; @@ -47,7 +47,7 @@ */ public class GroupReduceOperator extends SingleInputUdfOperator> { - private final GroupReducible function; + private final GroupReduceFunction function; private final Grouping grouper; @@ -62,7 +62,7 @@ public class GroupReduceOperator extends SingleInputUdfOperator input, GroupReducible function) { + public GroupReduceOperator(DataSet input, GroupReduceFunction function) { super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType())); this.function = function; @@ -77,7 +77,7 @@ public GroupReduceOperator(DataSet input, GroupReducible function) * @param input The grouped input to be processed group-wise by the groupReduce function. * @param function The user-defined GroupReduce function. */ - public GroupReduceOperator(Grouping input, GroupReducible function) { + public GroupReduceOperator(Grouping input, GroupReduceFunction function) { super(input != null ? input.getDataSet() : null, TypeExtractor.getGroupReduceReturnTypes(function, input.getDataSet().getType())); this.function = function; @@ -89,8 +89,8 @@ public GroupReduceOperator(Grouping input, GroupReducible function) } private void checkCombinability() { - if (function instanceof FlatCombinable && - function.getClass().getAnnotation(GroupReduceFunction.Combinable.class) != null) { + if (function instanceof FlatCombineFunction && + function.getClass().getAnnotation(RichGroupReduceFunction.Combinable.class) != null) { this.combinable = true; } } @@ -106,7 +106,7 @@ public boolean isCombinable() { public void setCombinable(boolean combinable) { // sanity check that the function is a subclass of the combine interface - if (combinable && !(function instanceof FlatCombinable)) { + if (combinable && !(function instanceof FlatCombineFunction)) { throw new IllegalArgumentException("The function does not implement the combine interface."); } @@ -122,8 +122,8 @@ public void setCombinable(boolean combinable) { if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); po.setCombinable(combinable); // set input @@ -149,8 +149,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getResultType()); - GroupReduceOperatorBase> po = - new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + GroupReduceOperatorBase> po = + new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setCombinable(combinable); po.setInput(input); @@ -182,7 +182,7 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { // -------------------------------------------------------------------------------------------- private static PlanUnwrappingReduceGroupOperator translateSelectorFunctionReducer( - Keys.SelectorFunctionKeys rawKeys, GroupReducible function, + Keys.SelectorFunctionKeys rawKeys, GroupReduceFunction function, TypeInformation inputType, TypeInformation outputType, String name, Operator input, boolean combinable) { @@ -195,7 +195,7 @@ private static PlanUnwrappingReduceGroupOperator transl PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); - MapOperatorBase, Mappable>> mapper = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, MapFunction>> mapper = new MapOperatorBase, MapFunction>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); reducer.setInput(mapper); mapper.setInput(input); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java index 37e74ef2c9caa..3223f4d5516ea 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java @@ -26,8 +26,8 @@ * Grouping is an intermediate step for a transformation on a grouped DataSet.
* The following transformation can be applied on Grouping: *

    - *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.ReduceFunction)},
  • - *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.GroupReduceFunction)}, and
  • + *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.RichReduceFunction)},
  • + *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)}, and
  • *
  • {@link UnsortedGrouping#aggregate(org.apache.flink.api.java.aggregation.Aggregations, int)}.
  • *
* diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index c75a3bb2f52ca..a07a15704e4cc 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -22,9 +22,9 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.FlatJoinable; -import org.apache.flink.api.common.functions.Joinable; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; @@ -34,7 +34,7 @@ import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.java.functions.FlatJoinFunction; +import org.apache.flink.api.java.functions.RichFlatJoinFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; @@ -152,12 +152,12 @@ protected JoinHint getJoinHint() { * @param The type of the second input DataSet of the Join transformation. * @param The type of the result of the Join transformation. * - * @see org.apache.flink.api.java.functions.FlatJoinFunction + * @see org.apache.flink.api.java.functions.RichFlatJoinFunction * @see DataSet */ public static class EquiJoin extends JoinOperator { - private final FlatJoinable function; + private final FlatJoinFunction function; @SuppressWarnings("unused") private boolean preserve1; @@ -165,7 +165,7 @@ public static class EquiJoin extends JoinOperator { private boolean preserve2; protected EquiJoin(DataSet input1, DataSet input2, - Keys keys1, Keys keys2, FlatJoinable function, + Keys keys1, Keys keys2, FlatJoinFunction function, TypeInformation returnType, JoinHint hint) { super(input1, input2, keys1, keys2, returnType, hint); @@ -184,7 +184,7 @@ protected EquiJoin(DataSet input1, DataSet input2, } protected EquiJoin(DataSet input1, DataSet input2, - Keys keys1, Keys keys2, FlatJoinable generatedFunction, Joinable function, + Keys keys1, Keys keys2, FlatJoinFunction generatedFunction, JoinFunction function, TypeInformation returnType, JoinHint hint) { super(input1, input2, keys1, keys2, returnType, hint); @@ -262,8 +262,8 @@ else if ((super.keys1 instanceof Keys.FieldPositionKeys int[] logicalKeyPositions1 = super.keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = super.keys2.computeLogicalKeyPositions(); - JoinOperatorBase> po = - new JoinOperatorBase>(function, + JoinOperatorBase> po = + new JoinOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); @@ -322,7 +322,7 @@ else if (super.keys1 instanceof Keys.SelectorFunctionKeys private static PlanUnwrappingJoinOperator translateSelectorFunctionJoin( Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, - FlatJoinable function, + FlatJoinFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -337,10 +337,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); join.setFirstInput(keyMapper1); @@ -357,7 +357,7 @@ private static PlanUnwrappingJoinOperator trans private static PlanUnwrappingJoinOperator translateSelectorFunctionJoinRight( int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, - FlatJoinable function, + FlatJoinFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -374,10 +374,10 @@ private static PlanUnwrappingJoinOperator trans final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -395,7 +395,7 @@ private static PlanUnwrappingJoinOperator trans private static PlanUnwrappingJoinOperator translateSelectorFunctionJoinLeft( Keys.SelectorFunctionKeys rawKeys1, int[] logicalKeyPositions2, - FlatJoinable function, + FlatJoinFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -412,10 +412,10 @@ private static PlanUnwrappingJoinOperator trans final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - final MapOperatorBase, Mappable>> keyMapper1 = - new MapOperatorBase, Mappable>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); - final MapOperatorBase, Mappable>> keyMapper2 = - new MapOperatorBase, Mappable>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); + final MapOperatorBase, MapFunction>> keyMapper1 = + new MapOperatorBase, MapFunction>>(extractor1, new UnaryOperatorInformation>(inputType1, typeInfoWithKey1), "Key Extractor 1"); + final MapOperatorBase, MapFunction>> keyMapper2 = + new MapOperatorBase, MapFunction>>(extractor2, new UnaryOperatorInformation>(inputType2, typeInfoWithKey2), "Key Extractor 2"); final PlanUnwrappingJoinOperator join = new PlanUnwrappingJoinOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -448,22 +448,22 @@ protected DefaultJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { super(input1, input2, keys1, keys2, - (FlatJoinFunction>) new DefaultFlatJoinFunction(), + (RichFlatJoinFunction>) new DefaultFlatJoinFunction(), new TupleTypeInfo>(input1.getType(), input2.getType()), hint); } /** - * Finalizes a Join transformation by applying a {@link org.apache.flink.api.java.functions.FlatJoinFunction} to each pair of joined elements.
+ * Finalizes a Join transformation by applying a {@link org.apache.flink.api.java.functions.RichFlatJoinFunction} to each pair of joined elements.
* Each JoinFunction call returns exactly one element. * * @param function The JoinFunction that is called for each pair of joined elements. * @return An EquiJoin that represents the joined result DataSet * - * @see org.apache.flink.api.java.functions.FlatJoinFunction + * @see org.apache.flink.api.java.functions.RichFlatJoinFunction * @see org.apache.flink.api.java.operators.JoinOperator.EquiJoin * @see DataSet */ - public EquiJoin with(FlatJoinable function) { + public EquiJoin with(FlatJoinFunction function) { if (function == null) { throw new NullPointerException("Join function must not be null."); } @@ -474,23 +474,23 @@ public EquiJoin with(FlatJoinable function) { return new EquiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), function, returnType, getJoinHint()); } - public EquiJoin with (Joinable function) { + public EquiJoin with (JoinFunction function) { if (function == null) { throw new NullPointerException("Join function must not be null."); } if (FunctionUtils.isSerializedLambdaFunction(function)) { throw new UnsupportedLambdaExpressionException(); } - FlatJoinable generatedFunction = new WrappingFlatJoinFunction(function); + FlatJoinFunction generatedFunction = new WrappingFlatJoinFunction(function); TypeInformation returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type()); return new EquiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint()); } - private static class WrappingFlatJoinFunction extends WrappingFunction> implements FlatJoinable { + private static class WrappingFlatJoinFunction extends WrappingFunction> implements FlatJoinFunction { private static final long serialVersionUID = 1L; - private WrappingFlatJoinFunction(Joinable wrappedFunction) { + private WrappingFlatJoinFunction(JoinFunction wrappedFunction) { super(wrappedFunction); } @@ -890,7 +890,7 @@ protected DefaultJoin createJoinOperator(Keys keys2) { // default join functions // -------------------------------------------------------------------------------------------- - public static final class DefaultFlatJoinFunction extends FlatJoinFunction> { + public static final class DefaultFlatJoinFunction extends RichFlatJoinFunction> { private static final long serialVersionUID = 1L; private final Tuple2 outTuple = new Tuple2(); @@ -903,7 +903,7 @@ public void join(T1 first, T2 second, Collector> out) throws Excep } } - public static final class ProjectFlatJoinFunction extends FlatJoinFunction { + public static final class ProjectFlatJoinFunction extends RichFlatJoinFunction { private static final long serialVersionUID = 1L; @@ -958,7 +958,7 @@ public void join(T1 in1, T2 in2, Collector out) { } } - public static final class LeftSemiFlatJoinFunction extends FlatJoinFunction { + public static final class LeftSemiFlatJoinFunction extends RichFlatJoinFunction { private static final long serialVersionUID = 1L; @@ -971,7 +971,7 @@ public void join (T1 left, T2 right, Collector out) { } } - public static final class RightSemiFlatJoinFunction extends FlatJoinFunction { + public static final class RightSemiFlatJoinFunction extends RichFlatJoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java index ef95ab0c1235c..eccdeeca09fdf 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; @@ -33,14 +33,14 @@ * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. * - * @see Mappable + * @see org.apache.flink.api.common.functions.MapFunction */ public class MapOperator extends SingleInputUdfOperator> { - protected final Mappable function; + protected final MapFunction function; - public MapOperator(DataSet input, Mappable function) { + public MapOperator(DataSet input, MapFunction function) { super(input, TypeExtractor.getMapReturnTypes(function, input.getType())); @@ -49,11 +49,11 @@ public MapOperator(DataSet input, Mappable function) { } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator - MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); + MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index 5d63b35089d14..dd5a3bda3f480 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -20,7 +20,7 @@ import java.util.Arrays; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; @@ -51,7 +51,7 @@ public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo retur } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection " + Arrays.toString(fields); // create operator PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java index 776c8e5b2787a..13a6c912e5b9a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.Mappable; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; @@ -39,11 +39,11 @@ * * @param The type of the data set reduced by the operator. * - * @see Reducible + * @see org.apache.flink.api.common.functions.ReduceFunction */ public class ReduceOperator extends SingleInputUdfOperator> { - private final Reducible function; + private final ReduceFunction function; private final Grouping grouper; @@ -54,7 +54,7 @@ public class ReduceOperator extends SingleInputUdfOperator input, Reducible function) { + public ReduceOperator(DataSet input, ReduceFunction function) { super(input, input.getType()); this.function = function; @@ -64,7 +64,7 @@ public ReduceOperator(DataSet input, Reducible function) { } - public ReduceOperator(Grouping input, Reducible function) { + public ReduceOperator(Grouping input, ReduceFunction function) { super(input.getDataSet(), input.getDataSet().getType()); this.function = function; @@ -82,8 +82,8 @@ public ReduceOperator(Grouping input, Reducible function) { if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, new int[0], name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, new int[0], name); // set input po.setInput(input); @@ -108,8 +108,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // reduce with field positions int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(getInputType(), getInputType()); - ReduceOperatorBase> po = - new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); + ReduceOperatorBase> po = + new ReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); // set input po.setInput(input); @@ -127,7 +127,7 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || // -------------------------------------------------------------------------------------------- private static MapOperatorBase, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys rawKeys, - Reducible function, TypeInformation inputType, String name, Operator input, int dop) + ReduceFunction function, TypeInformation inputType, String name, Operator input, int dop) { @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys = (Keys.SelectorFunctionKeys) rawKeys; @@ -138,8 +138,8 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys || PlanUnwrappingReduceOperator reducer = new PlanUnwrappingReduceOperator(function, keys, name, inputType, typeInfoWithKey); - MapOperatorBase, Mappable>> keyExtractingMap = new MapOperatorBase, Mappable>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); - MapOperatorBase, T, Mappable, T>> keyRemovingMap = new MapOperatorBase, T, Mappable, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); + MapOperatorBase, MapFunction>> keyExtractingMap = new MapOperatorBase, MapFunction>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor"); + MapOperatorBase, T, MapFunction, T>> keyRemovingMap = new MapOperatorBase, T, MapFunction, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor"); keyExtractingMap.setInput(input); reducer.setInput(keyExtractingMap); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java index fa2c1aadd657c..dcdbed4c8ab9b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java @@ -35,8 +35,8 @@ /** * The SingleInputUdfOperator is the base class of all unary operators that execute * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that - * have one input (such as {@link org.apache.flink.api.java.functions.MapFunction} or - * {@link org.apache.flink.api.java.functions.ReduceFunction}). + * have one input (such as {@link org.apache.flink.api.java.functions.RichMapFunction} or + * {@link org.apache.flink.api.java.functions.RichReduceFunction}). *

* This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 578641aa29362..97b2417780a8a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -21,10 +21,9 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; @@ -33,7 +32,7 @@ * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.
* The following transformation can be applied on sorted groups: *

    - *
  • {@link SortedGrouping#reduceGroup(GroupReduceFunction)},
  • + *
  • {@link SortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)},
  • *
* * @param The type of the elements of the sorted and grouped DataSet. @@ -68,18 +67,18 @@ protected Order[] getGroupSortOrders() { /** * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.
- * The transformation calls a {@link GroupReduceFunction} for each group of the DataSet. + * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} for each group of the DataSet. * A GroupReduceFunction can iterate over all elements of a group and emit any * number of output elements including none. * * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. * @return A GroupReduceOperator that represents the reduced DataSet. * - * @see GroupReduceFunction + * @see org.apache.flink.api.java.functions.RichGroupReduceFunction * @see GroupReduceOperator * @see DataSet */ - public GroupReduceOperator reduceGroup(GroupReducible reducer) { + public GroupReduceOperator reduceGroup(GroupReduceFunction reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java index a85ca3faf61ca..f347fef9f7c38 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java @@ -35,8 +35,8 @@ /** * The TwoInputUdfOperator is the base class of all binary operators that execute * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that - * have two inputs (such as {@link org.apache.flink.api.java.functions.JoinFunction} or - * {@link org.apache.flink.api.java.functions.CoGroupFunction}). + * have two inputs (such as {@link org.apache.flink.api.java.functions.RichJoinFunction} or + * {@link org.apache.flink.api.java.functions.RichCoGroupFunction}). *

* This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java index 583f130dc0186..bf33f4e96b94c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java @@ -27,8 +27,8 @@ /** * This interface marks operators as operators that execute user-defined functions (UDFs), such as - * {@link org.apache.flink.api.java.functions.MapFunction}, {@link org.apache.flink.api.java.functions.ReduceFunction}, - * or {@link org.apache.flink.api.java.functions.CoGroupFunction}. + * {@link org.apache.flink.api.java.functions.RichMapFunction}, {@link org.apache.flink.api.java.functions.RichReduceFunction}, + * or {@link org.apache.flink.api.java.functions.RichCoGroupFunction}. * The UDF operators stand in contrast to operators that execute built-in operations, like aggregations. */ public interface UdfOperator> { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 9985c04cbe12c..9e71ba0e08d66 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -18,13 +18,11 @@ package org.apache.flink.api.java.operators; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.ReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException; @@ -94,18 +92,18 @@ public AggregateOperator min (int field) { /** * Applies a Reduce transformation on a grouped {@link DataSet}.
- * For each group, the transformation consecutively calls a {@link ReduceFunction} + * For each group, the transformation consecutively calls a {@link org.apache.flink.api.java.functions.RichReduceFunction} * until only a single element for each group remains. * A ReduceFunction combines two elements into one new element of the same type. * * @param reducer The ReduceFunction that is applied on each group of the DataSet. * @return A ReduceOperator that represents the reduced DataSet. * - * @see ReduceFunction + * @see org.apache.flink.api.java.functions.RichReduceFunction * @see ReduceOperator * @see DataSet */ - public ReduceOperator reduce(Reducible reducer) { + public ReduceOperator reduce(ReduceFunction reducer) { if (reducer == null) { throw new NullPointerException("Reduce function must not be null."); } @@ -114,18 +112,18 @@ public ReduceOperator reduce(Reducible reducer) { /** * Applies a GroupReduce transformation on a grouped {@link DataSet}.
- * The transformation calls a {@link GroupReduceFunction} for each group of the DataSet. + * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} for each group of the DataSet. * A GroupReduceFunction can iterate over all elements of a group and emit any * number of output elements including none. * * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. * @return A GroupReduceOperator that represents the reduced DataSet. * - * @see GroupReduceFunction + * @see org.apache.flink.api.java.functions.RichGroupReduceFunction * @see GroupReduceOperator * @see DataSet */ - public GroupReduceOperator reduceGroup(GroupReducible reducer) { + public GroupReduceOperator reduceGroup(GroupReduceFunction reducer) { if (reducer == null) { throw new NullPointerException("GroupReduce function must not be null."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java index aea99e3266217..c7f65f0f519da 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java @@ -19,11 +19,11 @@ package org.apache.flink.api.java.operators.translation; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; -public final class KeyExtractingMapper extends MapFunction> { +public final class KeyExtractingMapper extends RichMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java index 52cbcd3737988..a6cd8372f6e99 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java @@ -18,11 +18,11 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; -public final class KeyRemovingMapper extends MapFunction, T> { +public final class KeyRemovingMapper extends RichMapFunction, T> { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java index 5795f37a36f49..8ac2d01938277 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java @@ -18,27 +18,27 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.Filterable; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FilterOperatorBase; import org.apache.flink.types.TypeInformation; import org.apache.flink.util.Collector; -public class PlanFilterOperator extends FilterOperatorBase> { +public class PlanFilterOperator extends FilterOperatorBase> { - public PlanFilterOperator(Filterable udf, String name, TypeInformation type) { + public PlanFilterOperator(FilterFunction udf, String name, TypeInformation type) { super(new FlatMapFilter(udf), new UnaryOperatorInformation(type, type), name); } - public static final class FlatMapFilter extends WrappingFunction> - implements FlatMappable + public static final class FlatMapFilter extends WrappingFunction> + implements FlatMapFunction { private static final long serialVersionUID = 1L; - private FlatMapFilter(Filterable wrapped) { + private FlatMapFilter(FilterFunction wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java index 7044637de4d3f..4de73119fe2a6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java @@ -19,13 +19,13 @@ package org.apache.flink.api.java.operators.translation; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.types.TypeInformation; -public class PlanProjectOperator extends MapOperatorBase> { +public class PlanProjectOperator extends MapOperatorBase> { public PlanProjectOperator(int[] fields, String name, TypeInformation inType, TypeInformation outType) { super(new MapProjector(fields, outType.createSerializer().createInstance()), new UnaryOperatorInformation(inType, outType), name); @@ -33,7 +33,7 @@ public PlanProjectOperator(int[] fields, String name, TypeInformation inType, public static final class MapProjector extends AbstractRichFunction - implements Mappable + implements MapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java index 5c004a89262a9..89290f0165f84 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java @@ -20,7 +20,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; import org.apache.flink.api.java.operators.Keys; @@ -29,10 +29,10 @@ import org.apache.flink.util.Collector; public class PlanUnwrappingCoGroupOperator - extends CoGroupOperatorBase, Tuple2, OUT, CoGroupable, Tuple2, OUT>> + extends CoGroupOperatorBase, Tuple2, OUT, CoGroupFunction, Tuple2, OUT>> { - public PlanUnwrappingCoGroupOperator(CoGroupable udf, + public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, Keys.SelectorFunctionKeys key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -41,7 +41,7 @@ public PlanUnwrappingCoGroupOperator(CoGroupable udf, key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingCoGroupOperator(CoGroupable udf, + public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, int[] key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -50,7 +50,7 @@ public PlanUnwrappingCoGroupOperator(CoGroupable udf, new int[]{0}, key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingCoGroupOperator(CoGroupable udf, + public PlanUnwrappingCoGroupOperator(CoGroupFunction udf, Keys.SelectorFunctionKeys key1, int[] key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -61,15 +61,15 @@ public PlanUnwrappingCoGroupOperator(CoGroupable udf, // -------------------------------------------------------------------------------------------- - public static final class TupleUnwrappingCoGrouper extends WrappingFunction> - implements CoGroupable, Tuple2, OUT> + public static final class TupleUnwrappingCoGrouper extends WrappingFunction> + implements CoGroupFunction, Tuple2, OUT> { private static final long serialVersionUID = 1L; private final TupleUnwrappingIterator iter1; private final TupleUnwrappingIterator iter2; - private TupleUnwrappingCoGrouper(CoGroupable wrapped) { + private TupleUnwrappingCoGrouper(CoGroupFunction wrapped) { super(wrapped); this.iter1 = new TupleUnwrappingIterator(); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java index 1762fd91ceef8..73ea0043570ec 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.java.operators.Keys; @@ -27,10 +27,10 @@ import org.apache.flink.util.Collector; public class PlanUnwrappingJoinOperator - extends JoinOperatorBase, Tuple2, OUT, FlatJoinable, Tuple2, OUT>> + extends JoinOperatorBase, Tuple2, OUT, FlatJoinFunction, Tuple2, OUT>> { - public PlanUnwrappingJoinOperator(FlatJoinable udf, + public PlanUnwrappingJoinOperator(FlatJoinFunction udf, Keys.SelectorFunctionKeys key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -39,7 +39,7 @@ public PlanUnwrappingJoinOperator(FlatJoinable udf, key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingJoinOperator(FlatJoinable udf, + public PlanUnwrappingJoinOperator(FlatJoinFunction udf, int[] key1, Keys.SelectorFunctionKeys key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -48,7 +48,7 @@ public PlanUnwrappingJoinOperator(FlatJoinable udf, new int[]{0}, key2.computeLogicalKeyPositions(), name); } - public PlanUnwrappingJoinOperator(FlatJoinable udf, + public PlanUnwrappingJoinOperator(FlatJoinFunction udf, Keys.SelectorFunctionKeys key1, int[] key2, String name, TypeInformation type, TypeInformation> typeInfoWithKey1, TypeInformation> typeInfoWithKey2) { @@ -58,13 +58,13 @@ public PlanUnwrappingJoinOperator(FlatJoinable udf, } public static final class TupleUnwrappingJoiner - extends WrappingFunction> - implements FlatJoinable, Tuple2, OUT> + extends WrappingFunction> + implements FlatJoinFunction, Tuple2, OUT> { private static final long serialVersionUID = 1L; - private TupleUnwrappingJoiner(FlatJoinable wrapped) { + private TupleUnwrappingJoiner(FlatJoinFunction wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java index 1038095aacf9c..5e8045578a7f9 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java @@ -20,11 +20,11 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.TypeInformation; @@ -34,12 +34,12 @@ * A reduce operator that takes 2-tuples (key-value pairs), and applies the group reduce operation only * on the unwrapped values. */ -public class PlanUnwrappingReduceGroupOperator extends GroupReduceOperatorBase, OUT, GroupReducible,OUT>> { +public class PlanUnwrappingReduceGroupOperator extends GroupReduceOperatorBase, OUT, GroupReduceFunction,OUT>> { - public PlanUnwrappingReduceGroupOperator(GroupReducible udf, Keys.SelectorFunctionKeys key, String name, + public PlanUnwrappingReduceGroupOperator(GroupReduceFunction udf, Keys.SelectorFunctionKeys key, String name, TypeInformation outType, TypeInformation> typeInfoWithKey, boolean combinable) { - super(combinable ? new TupleUnwrappingFlatCombinableGroupReducer((GroupReduceFunction) udf) : new TupleUnwrappingNonCombinableGroupReducer(udf), + super(combinable ? new TupleUnwrappingFlatCombinableGroupReducer((RichGroupReduceFunction) udf) : new TupleUnwrappingNonCombinableGroupReducer(udf), new UnaryOperatorInformation, OUT>(typeInfoWithKey, outType), key.computeLogicalKeyPositions(), name); super.setCombinable(combinable); @@ -47,9 +47,9 @@ public PlanUnwrappingReduceGroupOperator(GroupReducible udf, Keys.Selec // -------------------------------------------------------------------------------------------- - @GroupReduceFunction.Combinable - public static final class TupleUnwrappingFlatCombinableGroupReducer extends WrappingFunction> - implements GroupReducible, OUT>, FlatCombinable> + @RichGroupReduceFunction.Combinable + public static final class TupleUnwrappingFlatCombinableGroupReducer extends WrappingFunction> + implements GroupReduceFunction, OUT>, FlatCombineFunction> { private static final long serialVersionUID = 1L; @@ -57,7 +57,7 @@ public static final class TupleUnwrappingFlatCombinableGroupReducer private TupleUnwrappingIterator iter; private TupleWrappingCollector coll; - private TupleUnwrappingFlatCombinableGroupReducer(GroupReduceFunction wrapped) { + private TupleUnwrappingFlatCombinableGroupReducer(RichGroupReduceFunction wrapped) { super(wrapped); this.iter = new TupleUnwrappingIterator(); this.coll = new TupleWrappingCollector(this.iter); @@ -83,15 +83,15 @@ public String toString() { } } - public static final class TupleUnwrappingNonCombinableGroupReducer extends WrappingFunction> - implements GroupReducible, OUT> + public static final class TupleUnwrappingNonCombinableGroupReducer extends WrappingFunction> + implements GroupReduceFunction, OUT> { private static final long serialVersionUID = 1L; private final TupleUnwrappingIterator iter; - private TupleUnwrappingNonCombinableGroupReducer(GroupReducible wrapped) { + private TupleUnwrappingNonCombinableGroupReducer(GroupReduceFunction wrapped) { super(wrapped); this.iter = new TupleUnwrappingIterator(); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java index 8abbae3ef069e..4da981c772c71 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java @@ -18,7 +18,7 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; import org.apache.flink.api.java.operators.Keys; @@ -30,21 +30,21 @@ * A reduce operator that takes 2-tuples (key-value pairs), and applies the reduce operation only * on the unwrapped values. */ -public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, Reducible>> { +public class PlanUnwrappingReduceOperator extends ReduceOperatorBase, ReduceFunction>> { - public PlanUnwrappingReduceOperator(Reducible udf, Keys.SelectorFunctionKeys key, String name, + public PlanUnwrappingReduceOperator(ReduceFunction udf, Keys.SelectorFunctionKeys key, String name, TypeInformation type, TypeInformation> typeInfoWithKey) { super(new ReduceWrapper(udf), new UnaryOperatorInformation, Tuple2>(typeInfoWithKey, typeInfoWithKey), key.computeLogicalKeyPositions(), name); } - public static final class ReduceWrapper extends WrappingFunction> - implements Reducible> + public static final class ReduceWrapper extends WrappingFunction> + implements ReduceFunction> { private static final long serialVersionUID = 1L; - private ReduceWrapper(Reducible wrapped) { + private ReduceWrapper(ReduceFunction wrapped) { super(wrapped); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java index a915d1cecc544..ecac77529e0d6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java @@ -18,12 +18,12 @@ package org.apache.flink.api.java.operators.translation; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; -public final class TupleKeyExtractingMapper extends MapFunction> { +public final class TupleKeyExtractingMapper extends RichMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java index c513d9971ef4d..fa0ca119424cb 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java @@ -22,14 +22,13 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.CoGroupable; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; /** * The CoGroupFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CoGroupOperator}. */ -public abstract class CoGroupFunction extends AbstractRichFunction implements CoGroupable { +public abstract class CoGroupFunction extends AbstractRichFunction implements org.apache.flink.api.common.functions.CoGroupFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java index 093fc36499706..c4587fde8151f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CrossFunction.java @@ -20,13 +20,12 @@ package org.apache.flink.api.java.record.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Crossable; import org.apache.flink.types.Record; /** * The CrossFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CrossOperator}. */ -public abstract class CrossFunction extends AbstractRichFunction implements Crossable { +public abstract class CrossFunction extends AbstractRichFunction implements org.apache.flink.api.common.functions.CrossFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java index 29f9e4034d703..dce24a3323f9c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/JoinFunction.java @@ -20,7 +20,7 @@ package org.apache.flink.api.java.record.functions; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -28,7 +28,7 @@ * The JoinFunction must implementation by functions of a {@link org.apache.flink.api.java.operators.JoinOperator}. * It resembles an equality join of both inputs on their key fields. */ -public abstract class JoinFunction extends AbstractRichFunction implements FlatJoinable { +public abstract class JoinFunction extends AbstractRichFunction implements FlatJoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java index d144936b93246..073b11adb4c06 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java @@ -22,8 +22,8 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -31,7 +31,7 @@ * The ReduceFunction must be extended to provide a reducer implementation, as invoked by a * {@link org.apache.flink.api.java.operators.ReduceOperator}. */ -public abstract class ReduceFunction extends AbstractRichFunction implements GroupReducible, FlatCombinable { +public abstract class ReduceFunction extends AbstractRichFunction implements GroupReduceFunction, FlatCombineFunction { private static final long serialVersionUID = 1L; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index c80e64881168b..e8ee0bb6c1fd2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -31,13 +31,13 @@ import java.util.Set; import org.apache.commons.lang3.Validate; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.common.functions.Crossable; -import org.apache.flink.api.common.functions.FlatJoinable; -import org.apache.flink.api.common.functions.FlatMappable; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.common.functions.Mappable; -import org.apache.flink.api.common.functions.Joinable; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.java.functions.InvalidTypesException; import org.apache.flink.api.java.functions.KeySelector; @@ -61,75 +61,75 @@ private TypeExtractor() { // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") - public static TypeInformation getMapReturnTypes(Mappable mapInterface, TypeInformation inType) { - validateInputType(Mappable.class, mapInterface.getClass(), 0, inType); + public static TypeInformation getMapReturnTypes(MapFunction mapInterface, TypeInformation inType) { + validateInputType(MapFunction.class, mapInterface.getClass(), 0, inType); if(mapInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) mapInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(Mappable.class, mapInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(MapFunction.class, mapInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getFlatMapReturnTypes(FlatMappable flatMapInterface, TypeInformation inType) { - validateInputType(FlatMappable.class, flatMapInterface.getClass(), 0, inType); + public static TypeInformation getFlatMapReturnTypes(FlatMapFunction flatMapInterface, TypeInformation inType) { + validateInputType(FlatMapFunction.class, flatMapInterface.getClass(), 0, inType); if(flatMapInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) flatMapInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(FlatMappable.class, flatMapInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(FlatMapFunction.class, flatMapInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getGroupReduceReturnTypes(GroupReducible groupReduceInterface, + public static TypeInformation getGroupReduceReturnTypes(GroupReduceFunction groupReduceInterface, TypeInformation inType) { - validateInputType(GroupReducible.class, groupReduceInterface.getClass(), 0, inType); + validateInputType(GroupReduceFunction.class, groupReduceInterface.getClass(), 0, inType); if(groupReduceInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) groupReduceInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(GroupReducible.class, groupReduceInterface.getClass(), 1, inType, null); + return new TypeExtractor().privateCreateTypeInfo(GroupReduceFunction.class, groupReduceInterface.getClass(), 1, inType, null); } @SuppressWarnings("unchecked") - public static TypeInformation getJoinReturnTypes(FlatJoinable joinInterface, + public static TypeInformation getJoinReturnTypes(FlatJoinFunction joinInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(FlatJoinable.class, joinInterface.getClass(), 0, in1Type); - validateInputType(FlatJoinable.class, joinInterface.getClass(), 1, in2Type); + validateInputType(FlatJoinFunction.class, joinInterface.getClass(), 0, in1Type); + validateInputType(FlatJoinFunction.class, joinInterface.getClass(), 1, in2Type); if(joinInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) joinInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(FlatJoinable.class, joinInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(FlatJoinFunction.class, joinInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") - public static TypeInformation getJoinReturnTypes(Joinable joinInterface, + public static TypeInformation getJoinReturnTypes(JoinFunction joinInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(Joinable.class, joinInterface.getClass(), 0, in1Type); - validateInputType(Joinable.class, joinInterface.getClass(), 1, in2Type); + validateInputType(JoinFunction.class, joinInterface.getClass(), 0, in1Type); + validateInputType(JoinFunction.class, joinInterface.getClass(), 1, in2Type); if(joinInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) joinInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(Joinable.class, joinInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(JoinFunction.class, joinInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") - public static TypeInformation getCoGroupReturnTypes(CoGroupable coGroupInterface, + public static TypeInformation getCoGroupReturnTypes(CoGroupFunction coGroupInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(CoGroupable.class, coGroupInterface.getClass(), 0, in1Type); - validateInputType(CoGroupable.class, coGroupInterface.getClass(), 1, in2Type); + validateInputType(CoGroupFunction.class, coGroupInterface.getClass(), 0, in1Type); + validateInputType(CoGroupFunction.class, coGroupInterface.getClass(), 1, in2Type); if(coGroupInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) coGroupInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(CoGroupable.class, coGroupInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(CoGroupFunction.class, coGroupInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") - public static TypeInformation getCrossReturnTypes(Crossable crossInterface, + public static TypeInformation getCrossReturnTypes(CrossFunction crossInterface, TypeInformation in1Type, TypeInformation in2Type) { - validateInputType(Crossable.class, crossInterface.getClass(), 0, in1Type); - validateInputType(Crossable.class, crossInterface.getClass(), 1, in2Type); + validateInputType(CrossFunction.class, crossInterface.getClass(), 0, in1Type); + validateInputType(CrossFunction.class, crossInterface.getClass(), 1, in2Type); if(crossInterface instanceof ResultTypeQueryable) { return ((ResultTypeQueryable) crossInterface).getProducedType(); } - return new TypeExtractor().privateCreateTypeInfo(Crossable.class, crossInterface.getClass(), 2, in1Type, in2Type); + return new TypeExtractor().privateCreateTypeInfo(CrossFunction.class, crossInterface.getClass(), 2, in1Type, in2Type); } @SuppressWarnings("unchecked") diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java index 11595128c0eb8..474b0227400ce 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java @@ -28,8 +28,6 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; @@ -284,7 +282,7 @@ public void translateBinaryFunctionAnnotationTuples() { @ConstantFields("*") - public static class WildcardConstantMapper extends MapFunction { + public static class WildcardConstantMapper extends RichMapFunction { @Override public T map(T value) { @@ -293,7 +291,7 @@ public T map(T value) { } @ConstantFields("0->0;1->1;2->2") - public static class IndividualConstantMapper extends MapFunction, Tuple3> { + public static class IndividualConstantMapper extends RichMapFunction, Tuple3> { @Override public Tuple3 map(Tuple3 value) { @@ -302,7 +300,7 @@ public Tuple3 map(Tuple3 value) { } @ConstantFields("0") - public static class ZeroConstantMapper extends MapFunction { + public static class ZeroConstantMapper extends RichMapFunction { @Override public T map(T value) { @@ -312,7 +310,7 @@ public T map(T value) { @ConstantFieldsFirst("1 -> 0") @ConstantFieldsSecond("1 -> 1") - public static class ForwardingTupleJoin extends JoinFunction, Tuple2, Tuple2> { + public static class ForwardingTupleJoin extends RichJoinFunction, Tuple2, Tuple2> { @Override public Tuple2 join(Tuple2 first, Tuple2 second) { @@ -322,7 +320,7 @@ public Tuple2 join(Tuple2 first, Tuple2 second) { @ConstantFieldsFirst("0 -> 0") @ConstantFieldsSecond("0 -> 1") - public static class ForwardingBasicJoin extends JoinFunction> { + public static class ForwardingBasicJoin extends RichJoinFunction> { @Override public Tuple2 join(A first, B second) { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java index 30c8db14c0682..155bbd11cd0fd 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java @@ -35,9 +35,9 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichCoGroupFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.util.Collector; @@ -221,21 +221,21 @@ public void testRejectWhenSolutionSetKeysDontMatchCoGroup() { // -------------------------------------------------------------------------------------------- - public static class SolutionWorksetJoin extends JoinFunction, Tuple3, Tuple3> { + public static class SolutionWorksetJoin extends RichJoinFunction, Tuple3, Tuple3> { @Override public Tuple3 join(Tuple2 first, Tuple3 second){ return null; } } - public static class NextWorksetMapper extends MapFunction, Tuple2> { + public static class NextWorksetMapper extends RichMapFunction, Tuple2> { @Override public Tuple2 map(Tuple3 value) { return null; } } - public static class IdentityMapper extends MapFunction { + public static class IdentityMapper extends RichMapFunction { @Override public T map(T value) throws Exception { @@ -243,7 +243,7 @@ public T map(T value) throws Exception { } } - public static class SolutionWorksetCoGroup1 extends CoGroupFunction, Tuple3, Tuple3> { + public static class SolutionWorksetCoGroup1 extends RichCoGroupFunction, Tuple3, Tuple3> { @Override public void coGroup(Iterator> first, Iterator> second, @@ -251,7 +251,7 @@ public void coGroup(Iterator> first, Iterator, Tuple2, Tuple3> { + public static class SolutionWorksetCoGroup2 extends RichCoGroupFunction, Tuple2, Tuple3> { @Override public void coGroup(Iterator> second, Iterator> first, diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java index 9f6a6d8610206..8e457ce6a2e67 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java @@ -27,9 +27,7 @@ import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.ReduceFunction; -import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; -import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceOperator; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -53,7 +51,7 @@ public void translateNonGroupedReduce() { DataSet> initialData = getSourceDataSet(env); - initialData.reduce(new ReduceFunction>() { + initialData.reduce(new RichReduceFunction>() { public Tuple3 reduce(Tuple3 value1, Tuple3 value2) { return value1; } @@ -94,7 +92,7 @@ public void translateGroupedReduceNoMapper() { initialData .groupBy(2) - .reduce(new ReduceFunction>() { + .reduce(new RichReduceFunction>() { public Tuple3 reduce(Tuple3 value1, Tuple3 value2) { return value1; } @@ -141,7 +139,7 @@ public StringValue getKey(Tuple3 value) { return value.f1; } }) - .reduce(new ReduceFunction>() { + .reduce(new RichReduceFunction>() { public Tuple3 reduce(Tuple3 value1, Tuple3 value2) { return value1; } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java index ccf38494baa95..c6ad73ddff945 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java @@ -23,16 +23,16 @@ import java.io.IOException; import java.util.Iterator; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.CrossFunction; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichCoGroupFunction; +import org.apache.flink.api.java.functions.RichCrossFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.functions.InvalidTypesException; -import org.apache.flink.api.java.functions.FlatJoinFunction; +import org.apache.flink.api.java.functions.RichFlatJoinFunction; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; @@ -68,7 +68,7 @@ public class TypeExtractorTest { @Test public void testBasicType() { // use getGroupReduceReturnTypes() - GroupReduceFunction function = new GroupReduceFunction() { + RichGroupReduceFunction function = new RichGroupReduceFunction() { private static final long serialVersionUID = 1L; @Override @@ -107,7 +107,7 @@ public void readFields(DataInput in) throws IOException { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testWritableType() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -127,7 +127,7 @@ public MyWritable map(MyWritable value) throws Exception { @Test public void testTupleWithBasicTypes() throws Exception { // use getMapReturnTypes() - MapFunction function = new MapFunction, Tuple9>() { + RichMapFunction function = new RichMapFunction, Tuple9>() { private static final long serialVersionUID = 1L; @Override @@ -192,7 +192,7 @@ public Tuple9 function = new FlatMapFunction, Tuple1, Tuple2>, Tuple3, Tuple1, Tuple2>>() { + RichFlatMapFunction function = new RichFlatMapFunction, Tuple1, Tuple2>, Tuple3, Tuple1, Tuple2>>() { private static final long serialVersionUID = 1L; @Override @@ -247,7 +247,7 @@ public void flatMap(Tuple3, Tuple1, Tuple2> @Test public void testSubclassOfTuple() { // use getJoinReturnTypes() - FlatJoinFunction function = new FlatJoinFunction() { + RichFlatJoinFunction function = new RichFlatJoinFunction() { private static final long serialVersionUID = 1L; @Override @@ -295,7 +295,7 @@ public int getMyField2() { @Test public void testCustomType() { // use getCrossReturnTypes() - CrossFunction function = new CrossFunction() { + RichCrossFunction function = new RichCrossFunction() { private static final long serialVersionUID = 1L; @Override @@ -342,7 +342,7 @@ public CustomType(String myField1, int myField2) { @Test public void testTupleWithCustomType() { // use getMapReturnTypes() - MapFunction function = new MapFunction, Tuple2>() { + RichMapFunction function = new RichMapFunction, Tuple2>() { private static final long serialVersionUID = 1L; @Override @@ -412,7 +412,7 @@ public StringValue getKey(StringValue value) { @Test public void testTupleOfValues() { // use getMapReturnTypes() - MapFunction function = new MapFunction, Tuple2>() { + RichMapFunction function = new RichMapFunction, Tuple2>() { private static final long serialVersionUID = 1L; @Override @@ -451,7 +451,7 @@ public LongKeyValue(Long field1, V field2) { @Test public void testGenericsNotInSuperclass() { // use getMapReturnTypes() - MapFunction function = new MapFunction, LongKeyValue>() { + RichMapFunction function = new RichMapFunction, LongKeyValue>() { private static final long serialVersionUID = 1L; @Override @@ -494,7 +494,7 @@ public ChainedTwo(String field0, Long field1, V field2) { @Test public void testChainedGenericsNotInSuperclass() { // use TypeExtractor - MapFunction function = new MapFunction, ChainedTwo>() { + RichMapFunction function = new RichMapFunction, ChainedTwo>() { private static final long serialVersionUID = 1L; @Override @@ -536,7 +536,7 @@ public ChainedFour(String field0, Long field1, String field2) { @Test public void testGenericsInDirectSuperclass() { // use TypeExtractor - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -562,7 +562,7 @@ public ChainedThree map(ChainedThree value) throws Exception { @Test public void testGenericsNotInSuperclassWithNonGenericClassAtEnd() { // use TypeExtractor - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -587,7 +587,7 @@ public ChainedFour map(ChainedFour value) throws Exception { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testMissingTupleGenericsException() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -607,7 +607,7 @@ public Tuple2 map(String value) throws Exception { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testTupleSupertype() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -635,7 +635,7 @@ public SameTypeVariable(X field0, X field1) { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testSameGenericVariable() { - MapFunction function = new MapFunction, SameTypeVariable>() { + RichMapFunction function = new RichMapFunction, SameTypeVariable>() { private static final long serialVersionUID = 1L; @Override @@ -667,7 +667,7 @@ public Nested(V field0, Tuple2 field1) { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testNestedTupleGenerics() { - MapFunction function = new MapFunction, Nested>() { + RichMapFunction function = new RichMapFunction, Nested>() { private static final long serialVersionUID = 1L; @Override @@ -706,7 +706,7 @@ public Nested2(T field0, Tuple2, Nested> field1) @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testNestedTupleGenerics2() { - MapFunction function = new MapFunction, Nested2>() { + RichMapFunction function = new RichMapFunction, Nested2>() { private static final long serialVersionUID = 1L; @Override @@ -746,7 +746,7 @@ public Nested2 map(Nested2 value) throws Exception { @SuppressWarnings({ "unchecked", "rawtypes" }) @Test public void testFunctionWithMissingGenerics() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -776,7 +776,7 @@ public void testFunctionDependingOnInputAsSuperclass() { Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, ti); } - public class IdentityMapper extends MapFunction { + public class IdentityMapper extends RichMapFunction { private static final long serialVersionUID = 1L; @Override @@ -807,7 +807,7 @@ public void testFunctionDependingOnInputWithMissingInput() { } } - public class IdentityMapper2 extends MapFunction, T> { + public class IdentityMapper2 extends RichMapFunction, T> { private static final long serialVersionUID = 1L; @Override @@ -843,7 +843,7 @@ public void testFunctionDependingOnInputWithCustomTupleInput() { Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tti.getTypeAt(1)); } - public class IdentityMapper3 extends MapFunction { + public class IdentityMapper3 extends RichMapFunction { private static final long serialVersionUID = 1L; @Override @@ -916,7 +916,7 @@ public String map(String value) throws Exception { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testFunctionWithNoGenericSuperclass() { - MapFunction function = new Mapper2(); + RichMapFunction function = new Mapper2(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeInfoParser.parse("String")); @@ -924,7 +924,7 @@ public void testFunctionWithNoGenericSuperclass() { Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, ti); } - public class OneAppender extends MapFunction> { + public class OneAppender extends RichMapFunction> { private static final long serialVersionUID = 1L; public Tuple2 map(T value) { @@ -935,7 +935,7 @@ public Tuple2 map(T value) { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testFunctionDependingPartialOnInput() { - MapFunction function = new OneAppender() { + RichMapFunction function = new OneAppender() { private static final long serialVersionUID = 1L; }; @@ -955,7 +955,7 @@ public void testFunctionDependingPartialOnInput() { @Test public void testFunctionDependingPartialOnInput2() { - MapFunction function = new OneAppender(); + RichMapFunction function = new OneAppender(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, new ValueTypeInfo(DoubleValue.class)); @@ -971,7 +971,7 @@ public void testFunctionDependingPartialOnInput2() { Assert.assertEquals(Integer.class , tti.getTypeAt(1).getTypeClass()); } - public class FieldDuplicator extends MapFunction> { + public class FieldDuplicator extends RichMapFunction> { private static final long serialVersionUID = 1L; public Tuple2 map(T value) { @@ -981,7 +981,7 @@ public Tuple2 map(T value) { @Test public void testFunctionInputInOutputMultipleTimes() { - MapFunction function = new FieldDuplicator(); + RichMapFunction function = new FieldDuplicator(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, BasicTypeInfo.FLOAT_TYPE_INFO); @@ -994,7 +994,7 @@ public void testFunctionInputInOutputMultipleTimes() { @Test public void testFunctionInputInOutputMultipleTimes2() { - MapFunction, ?> function = new FieldDuplicator>(); + RichMapFunction, ?> function = new FieldDuplicator>(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, new TupleTypeInfo>( BasicTypeInfo.FLOAT_TYPE_INFO, BasicTypeInfo.FLOAT_TYPE_INFO)); @@ -1023,7 +1023,7 @@ public abstract class AbstractClass {} @Test public void testAbstractAndInterfaceTypesException() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1039,7 +1039,7 @@ public Testable map(String value) throws Exception { // good } - MapFunction function2 = new MapFunction() { + RichMapFunction function2 = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1059,7 +1059,7 @@ public AbstractClass map(String value) throws Exception { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testValueSupertypeException() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1080,7 +1080,7 @@ public Value map(StringValue value) throws Exception { @Test public void testBasicArray() { // use getCoGroupReturnTypes() - CoGroupFunction function = new CoGroupFunction() { + RichCoGroupFunction function = new RichCoGroupFunction() { private static final long serialVersionUID = 1L; @Override @@ -1107,7 +1107,7 @@ public void coGroup(Iterator first, Iterator second, Collect @Test public void testBasicArray2() { - MapFunction function = new IdentityMapper(); + RichMapFunction function = new IdentityMapper(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, BasicArrayTypeInfo.BOOLEAN_ARRAY_TYPE_INFO); @@ -1122,7 +1122,7 @@ public static class CustomArrayObject {} @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testCustomArray() { - MapFunction function = new MapFunction() { + RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1140,7 +1140,7 @@ public CustomArrayObject[] map(CustomArrayObject[] value) throws Exception { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testTupleArray() { - MapFunction function = new MapFunction[], Tuple2[]>() { + RichMapFunction function = new RichMapFunction[], Tuple2[]>() { private static final long serialVersionUID = 1L; @Override @@ -1167,7 +1167,7 @@ public class CustomArrayObject2 extends Tuple1 { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testCustomArrayWithTypeVariable() { - MapFunction[], ?> function = new IdentityMapper[]>(); + RichMapFunction[], ?> function = new IdentityMapper[]>(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeInfoParser.parse("Tuple1[]")); @@ -1178,7 +1178,7 @@ public void testCustomArrayWithTypeVariable() { Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, tti.getTypeAt(0)); } - public class GenericArrayClass extends MapFunction { + public class GenericArrayClass extends RichMapFunction { private static final long serialVersionUID = 1L; @Override @@ -1207,7 +1207,7 @@ public static class MyObject { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testParamertizedCustomObject() { - MapFunction function = new MapFunction, MyObject>() { + RichMapFunction function = new RichMapFunction, MyObject>() { private static final long serialVersionUID = 1L; @Override @@ -1242,7 +1242,7 @@ public void testFunctionDependingOnInputWithTupleInputWithTypeMismatch() { @Test public void testInputMismatchExceptions() { - MapFunction function = new MapFunction, String>() { + RichMapFunction function = new RichMapFunction, String>() { private static final long serialVersionUID = 1L; @Override @@ -1265,7 +1265,7 @@ public String map(Tuple2 value) throws Exception { // right } - MapFunction function2 = new MapFunction() { + RichMapFunction function2 = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1281,7 +1281,7 @@ public String map(StringValue value) throws Exception { // right } - MapFunction function3 = new MapFunction[], String>() { + RichMapFunction function3 = new RichMapFunction[], String>() { private static final long serialVersionUID = 1L; @Override @@ -1297,7 +1297,7 @@ public String map(Tuple1[] value) throws Exception { // right } - MapFunction function4 = new MapFunction() { + RichMapFunction function4 = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override @@ -1314,7 +1314,7 @@ public String map(Writable value) throws Exception { } } - public static class DummyFlatMapFunction extends FlatMapFunction, Tuple2> { + public static class DummyFlatMapFunction extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @Override @@ -1336,7 +1336,7 @@ public void testTypeErasureException() { } } - public static class MyQueryableMapper extends MapFunction implements ResultTypeQueryable { + public static class MyQueryableMapper extends RichMapFunction implements ResultTypeQueryable { private static final long serialVersionUID = 1L; @SuppressWarnings("unchecked") @@ -1359,7 +1359,7 @@ public void testResultTypeQueryable() { @Test public void testTupleWithPrimitiveArray() { - MapFunction> function = new MapFunction>() { + RichMapFunction> function = new RichMapFunction>() { private static final long serialVersionUID = 1L; @Override @@ -1383,7 +1383,7 @@ public Tuple9 mapInterface = new MapFunction() { + RichMapFunction mapInterface = new RichMapFunction() { @Override public void setRuntimeContext(RuntimeContext t) { @@ -1416,7 +1416,7 @@ public Boolean map(String record) throws Exception { @Test public void testInterface() { - Mappable mapInterface = new Mappable() { + MapFunction mapInterface = new MapFunction() { @Override public Boolean map(String record) throws Exception { return null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java index 88dfa580a5804..d7d63afb9bc08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -30,7 +30,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public abstract class AbstractCachedBuildSideMatchDriver extends MatchDriver implements ResettablePactDriver, OT> { +public abstract class AbstractCachedBuildSideMatchDriver extends MatchDriver implements ResettablePactDriver, OT> { private volatile JoinTaskIterator matchIterator; @@ -110,7 +110,7 @@ public void prepare() throws Exception { @Override public void run() throws Exception { - final FlatJoinable matchStub = this.taskContext.getStub(); + final FlatJoinFunction matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); if (buildSideIndex == 0) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java index 0765e55f0196a..0727d63ffbbb8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java @@ -21,8 +21,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.util.MutableToRegularIteratorWrapper; @@ -37,13 +37,13 @@ * The GroupReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the GroupReduceFunction. * - * @see org.apache.flink.api.common.functions.GroupReducible + * @see org.apache.flink.api.common.functions.GroupReduceFunction */ -public class AllGroupReduceDriver implements PactDriver, OT> { +public class AllGroupReduceDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(AllGroupReduceDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MutableObjectIterator input; @@ -54,7 +54,7 @@ public class AllGroupReduceDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GroupReducible.class; + final Class> clazz = (Class>) (Class) GroupReduceFunction.class; return clazz; } @@ -83,8 +83,8 @@ public void prepare() throws Exception { this.strategy = config.getDriverStrategy(); if (strategy == DriverStrategy.ALL_GROUP_COMBINE) { - if (!(this.taskContext.getStub() instanceof FlatCombinable)) { - throw new Exception("Using combiner on a UDF that does not implement the combiner interface " + FlatCombinable.class.getName()); + if (!(this.taskContext.getStub() instanceof FlatCombineFunction)) { + throw new Exception("Using combiner on a UDF that does not implement the combiner interface " + FlatCombineFunction.class.getName()); } } else if (strategy != DriverStrategy.ALL_GROUP_REDUCE) { @@ -105,13 +105,13 @@ public void run() throws Exception { // single UDF call with the single group if (inIter.hasNext()) { if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { - final GroupReducible reducer = this.taskContext.getStub(); + final GroupReduceFunction reducer = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); reducer.reduce(inIter, output); } else { @SuppressWarnings("unchecked") - final FlatCombinable combiner = (FlatCombinable) this.taskContext.getStub(); + final FlatCombineFunction combiner = (FlatCombineFunction) this.taskContext.getStub(); @SuppressWarnings("unchecked") final Collector output = (Collector) this.taskContext.getOutputCollector(); combiner.combine(inIter, output); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java index 741a9b4f65850..721f4f6ef8211 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -35,13 +35,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see org.apache.flink.api.common.functions.Reducible + * @see org.apache.flink.api.common.functions.ReduceFunction */ -public class AllReduceDriver implements PactDriver, T> { +public class AllReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(AllReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -52,7 +52,7 @@ public class AllReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -63,9 +63,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) Reducible.class; + final Class> clazz = (Class>) (Class) ReduceFunction.class; return clazz; } @@ -94,7 +94,7 @@ public void run() throws Exception { LOG.debug(this.taskContext.formatLogString("AllReduce preprocessing done. Running Reducer code.")); } - final Reducible stub = this.taskContext.getStub(); + final ReduceFunction stub = this.taskContext.getStub(); final MutableObjectIterator input = this.input; final TypeSerializer serializer = this.serializer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java index 8814c42c097e0..8ff0262f91a86 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -41,12 +41,12 @@ * * @see org.apache.flink.api.java.record.functions.CoGroupFunction */ -public class CoGroupDriver implements PactDriver, OT> { +public class CoGroupDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(CoGroupDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CoGroupTaskIterator coGroupIterator; // the iterator that does the actual cogroup @@ -56,7 +56,7 @@ public class CoGroupDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -69,9 +69,9 @@ public int getNumberOfInputs() { @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) CoGroupable.class; + final Class> clazz = (Class>) (Class) CoGroupFunction.class; return clazz; } @@ -122,7 +122,7 @@ public void prepare() throws Exception @Override public void run() throws Exception { - final CoGroupable coGroupStub = this.taskContext.getStub(); + final CoGroupFunction coGroupStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); final CoGroupTaskIterator coGroupIterator = this.coGroupIterator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java index 1bb471d2b0900..8761a2ea3d21b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -34,9 +34,9 @@ import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; -public class CoGroupWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { +public class CoGroupWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -53,7 +53,7 @@ public class CoGroupWithSolutionSetFirstDriver implements Resettab // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) CoGroupable.class; + final Class> clazz = (Class>) (Class) CoGroupFunction.class; return clazz; } @@ -123,7 +123,7 @@ public void prepare() { @Override public void run() throws Exception { - final CoGroupable coGroupStub = taskContext.getStub(); + final CoGroupFunction coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT1 buildSideRecord = solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java index 6afa690a80339..f2020c7a44be8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java @@ -21,7 +21,7 @@ import java.util.Iterator; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; @@ -34,9 +34,9 @@ import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; -public class CoGroupWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { +public class CoGroupWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -53,7 +53,7 @@ public class CoGroupWithSolutionSetSecondDriver implements Resetta // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -64,9 +64,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) CoGroupable.class; + final Class> clazz = (Class>) (Class) CoGroupFunction.class; return clazz; } @@ -123,7 +123,7 @@ public void prepare() { @Override public void run() throws Exception { - final CoGroupable coGroupStub = taskContext.getStub(); + final CoGroupFunction coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT2 buildSideRecord = solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java index 2f2bc548e9575..b68f24da4aea4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator; @@ -40,12 +40,12 @@ * * @see org.apache.flink.api.java.functions.CrossFunction */ -public class CrossDriver implements PactDriver, OT> { +public class CrossDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(CrossDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MemoryManager memManager; @@ -67,7 +67,7 @@ public class CrossDriver implements PactDriver @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -80,9 +80,9 @@ public int getNumberOfInputs() { @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) Crossable.class; + final Class> clazz = (Class>) (Class) CrossFunction.class; return clazz; } @@ -207,7 +207,7 @@ private void runBlockedOuterFirst() throws Exception { final T2 val2Reuse = serializer2.createInstance(); T2 val2Copy = serializer2.createInstance(); - final Crossable crosser = this.taskContext.getStub(); + final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -255,7 +255,7 @@ private void runBlockedOuterSecond() throws Exception { T2 val2; final T2 val2Reuse = serializer2.createInstance(); - final Crossable crosser = this.taskContext.getStub(); + final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -298,7 +298,7 @@ private void runStreamedOuterFirst() throws Exception { T2 val2; final T2 val2Reuse = serializer2.createInstance(); - final Crossable crosser = this.taskContext.getStub(); + final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks @@ -335,7 +335,7 @@ private void runStreamedOuterSecond() throws Exception { final T2 val2Reuse = serializer2.createInstance(); T2 val2Copy = serializer2.createInstance(); - final Crossable crosser = this.taskContext.getStub(); + final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); // for all blocks diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java index 6b88c5d79a0ac..130601b1da5dc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -31,20 +31,20 @@ * The MapTask creates an iterator over all key-value pairs of its input and hands that to the map() method * of the MapFunction. * - * @see org.apache.flink.api.common.functions.FlatMappable + * @see org.apache.flink.api.common.functions.FlatMapFunction * * @param The mapper's input data type. * @param The mapper's output data type. */ -public class FlatMapDriver implements PactDriver, OT> { +public class FlatMapDriver implements PactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -55,9 +55,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) FlatMappable.class; + final Class> clazz = (Class>) (Class) FlatMapFunction.class; return clazz; } @@ -75,7 +75,7 @@ public void prepare() { public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); - final FlatMappable function = this.taskContext.getStub(); + final FlatMapFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java index bc18b5b647c18..f786c5681fb59 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.memorymanager.MemoryManager; @@ -39,12 +39,12 @@ * * @param The data type consumed and produced by the combiner. */ -public class GroupReduceCombineDriver implements PactDriver, T> { +public class GroupReduceCombineDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(GroupReduceCombineDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private CloseableInputProvider input; @@ -57,7 +57,7 @@ public class GroupReduceCombineDriver implements PactDriver // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -68,9 +68,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) FlatCombinable.class; + final Class> clazz = (Class>) (Class) FlatCombineFunction.class; return clazz; } @@ -111,7 +111,7 @@ public void run() throws Exception { this.serializerFactory.getSerializer(), this.comparator); // cache references on the stack - final FlatCombinable stub = this.taskContext.getStub(); + final FlatCombineFunction stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 4b0526ce42ef0..960143d0e186f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -37,13 +37,13 @@ * The GroupReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the GroupReduceFunction. * - * @see org.apache.flink.api.common.functions.GroupReducible + * @see org.apache.flink.api.common.functions.GroupReduceFunction */ -public class GroupReduceDriver implements PactDriver, OT> { +public class GroupReduceDriver implements PactDriver, OT> { private static final Log LOG = LogFactory.getLog(GroupReduceDriver.class); - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private MutableObjectIterator input; @@ -56,7 +56,7 @@ public class GroupReduceDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -67,9 +67,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) GroupReducible.class; + final Class> clazz = (Class>) (Class) GroupReduceFunction.class; return clazz; } @@ -100,7 +100,7 @@ public void run() throws Exception { final KeyGroupedIterator iter = new KeyGroupedIterator(this.input, this.serializer, this.comparator); // cache references on the stack - final GroupReducible stub = this.taskContext.getStub(); + final GroupReduceFunction stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java index e90ca853cae47..342f307608689 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -32,9 +32,9 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public class JoinWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { +public class JoinWithSolutionSetFirstDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -50,7 +50,7 @@ public class JoinWithSolutionSetFirstDriver implements ResettableP // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -61,9 +61,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) FlatJoinable.class; + final Class> clazz = (Class>) (Class) FlatJoinFunction.class; return clazz; } @@ -126,7 +126,7 @@ public void prepare() { @Override public void run() throws Exception { - final FlatJoinable joinFunction = taskContext.getStub(); + final FlatJoinFunction joinFunction = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT1 buildSideRecord = this.solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java index 97ae2d857dff2..c38a81a8b2de6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -32,9 +32,9 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; -public class JoinWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { +public class JoinWithSolutionSetSecondDriver implements ResettablePactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private CompactingHashTable hashTable; @@ -50,7 +50,7 @@ public class JoinWithSolutionSetSecondDriver implements Resettable // -------------------------------------------------------------------------------------------- @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -61,9 +61,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) FlatJoinable.class; + final Class> clazz = (Class>) (Class) FlatJoinFunction.class; return clazz; } @@ -126,7 +126,7 @@ public void prepare() { @Override public void run() throws Exception { - final FlatJoinable joinFunction = taskContext.getStub(); + final FlatJoinFunction joinFunction = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); IT2 buildSideRecord = this.solutionSideRecord; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java index 7c1c72e4ea05b..89fbf4dbf0649 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -31,20 +31,20 @@ * The MapTask creates an iterator over all key-value pairs of its input and hands that to the map() method * of the MapFunction. * - * @see org.apache.flink.api.common.functions.Mappable + * @see org.apache.flink.api.common.functions.MapFunction * * @param The mapper's input data type. * @param The mapper's output data type. */ -public class MapDriver implements PactDriver, OT> { +public class MapDriver implements PactDriver, OT> { - private PactTaskContext, OT> taskContext; + private PactTaskContext, OT> taskContext; private volatile boolean running; @Override - public void setup(PactTaskContext, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -55,9 +55,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) Mappable.class; + final Class> clazz = (Class>) (Class) MapFunction.class; return clazz; } @@ -75,7 +75,7 @@ public void prepare() { public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); - final Mappable function = this.taskContext.getStub(); + final MapFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java index 05911037e041a..e205f1e723d13 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -42,13 +42,13 @@ * The MatchTask matches all pairs of records that share the same key and come from different inputs. Each pair of * matching records is handed to the match() method of the JoinFunction. * - * @see org.apache.flink.api.common.functions.FlatJoinable + * @see org.apache.flink.api.common.functions.FlatJoinFunction */ -public class MatchDriver implements PactDriver, OT> { +public class MatchDriver implements PactDriver, OT> { protected static final Log LOG = LogFactory.getLog(MatchDriver.class); - protected PactTaskContext, OT> taskContext; + protected PactTaskContext, OT> taskContext; private volatile JoinTaskIterator matchIterator; // the iterator that does the actual matching @@ -57,7 +57,7 @@ public class MatchDriver implements PactDriver, OT> context) { + public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; } @@ -68,9 +68,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) FlatJoinable.class; + final Class> clazz = (Class>) (Class) FlatJoinFunction.class; return clazz; } @@ -141,7 +141,7 @@ public void prepare() throws Exception{ @Override public void run() throws Exception { - final FlatJoinable matchStub = this.taskContext.getStub(); + final FlatJoinFunction matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); final JoinTaskIterator matchIterator = this.matchIterator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java index 5017ae487afba..87cea30c4394d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java @@ -24,7 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -44,7 +44,7 @@ * * @param The data type consumed and produced by the combiner. */ -public class ReduceCombineDriver implements PactDriver, T> { +public class ReduceCombineDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceCombineDriver.class); @@ -52,13 +52,13 @@ public class ReduceCombineDriver implements PactDriver, T> { private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32; - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private TypeSerializer serializer; private TypeComparator comparator; - private Reducible reducer; + private ReduceFunction reducer; private Collector output; @@ -75,7 +75,7 @@ public class ReduceCombineDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -86,9 +86,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) Reducible.class; + final Class> clazz = (Class>) (Class) ReduceFunction.class; return clazz; } @@ -168,7 +168,7 @@ private void sortAndCombine() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final Reducible function = this.reducer; + final ReduceFunction function = this.reducer; final Collector output = this.output; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java index d505bbd12ea92..9495cdcccd2b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java @@ -21,7 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -36,13 +36,13 @@ * The ReduceTask creates a iterator over all records from its input. The iterator returns all records grouped by their * key. The iterator is handed to the reduce() method of the ReduceFunction. * - * @see org.apache.flink.api.common.functions.Reducible + * @see org.apache.flink.api.common.functions.ReduceFunction */ -public class ReduceDriver implements PactDriver, T> { +public class ReduceDriver implements PactDriver, T> { private static final Log LOG = LogFactory.getLog(ReduceDriver.class); - private PactTaskContext, T> taskContext; + private PactTaskContext, T> taskContext; private MutableObjectIterator input; @@ -55,7 +55,7 @@ public class ReduceDriver implements PactDriver, T> { // ------------------------------------------------------------------------ @Override - public void setup(PactTaskContext, T> context) { + public void setup(PactTaskContext, T> context) { this.taskContext = context; this.running = true; } @@ -66,9 +66,9 @@ public int getNumberOfInputs() { } @Override - public Class> getStubType() { + public Class> getStubType() { @SuppressWarnings("unchecked") - final Class> clazz = (Class>) (Class) Reducible.class; + final Class> clazz = (Class>) (Class) ReduceFunction.class; return clazz; } @@ -101,7 +101,7 @@ public void run() throws Exception { final TypeSerializer serializer = this.serializer; final TypeComparator comparator = this.comparator; - final Reducible function = this.taskContext.getStub(); + final ReduceFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 204b60b866cf2..c8f217ce1debf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -526,8 +526,10 @@ protected void run() throws Exception { // modify accumulators.ll; if (this.stub != null) { // collect the counters from the stub - Map> accumulators = FunctionUtils.getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); - RegularPactTask.reportAndClearAccumulators(getEnvironment(), accumulators, this.chainedTasks); + if (FunctionUtils.getFunctionRuntimeContext(this.stub, this.runtimeUdfContext) != null) { + Map> accumulators = FunctionUtils.getFunctionRuntimeContext(this.stub, this.runtimeUdfContext).getAllAccumulators(); + RegularPactTask.reportAndClearAccumulators(getEnvironment(), accumulators, this.chainedTasks); + } } } catch (Exception ex) { @@ -583,9 +585,12 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - Map> chainedAccumulators = FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators(); - AccumulatorHelper.mergeInto(accumulators, chainedAccumulators); + if (FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null) != null) { + Map> chainedAccumulators = FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators(); + AccumulatorHelper.mergeInto(accumulators, chainedAccumulators); + } } // Don't report if the UDF didn't collect any accumulators @@ -608,7 +613,9 @@ protected static void reportAndClearAccumulators(Environment env, Map chainedTask : chainedTasks) { - AccumulatorHelper.resetAndClearAccumulators(FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators()); + if (FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null) != null) { + AccumulatorHelper.resetAndClearAccumulators(FunctionUtils.getFunctionRuntimeContext(chainedTask.getStub(), null).getAllAccumulators()); + } } } @@ -989,13 +996,13 @@ this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalSt e.getMessage() == null ? "." : ": " + e.getMessage(), e); } - if (!(localStub instanceof FlatCombinable)) { + if (!(localStub instanceof FlatCombineFunction)) { throw new IllegalStateException("Performing combining sort outside a reduce task!"); } @SuppressWarnings({ "rawtypes", "unchecked" }) CombiningUnilateralSortMerger cSorter = new CombiningUnilateralSortMerger( - (FlatCombinable) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum], + (FlatCombineFunction) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum), this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum), this.config.getSpillingThresholdInput(inputNum)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java index a0e4e06b0111d..db134a896473a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.operators.chaining; -import org.apache.flink.api.common.functions.FlatMappable; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; @@ -28,15 +28,15 @@ public class ChainedFlatMapDriver extends ChainedDriver { - private FlatMappable mapper; + private FlatMapFunction mapper; // -------------------------------------------------------------------------------------------- @Override public void setup(AbstractInvokable parent) { @SuppressWarnings("unchecked") - final FlatMappable mapper = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatMappable.class); + final FlatMapFunction mapper = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatMapFunction.class); this.mapper = mapper; FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java index a939995653abf..a7b1048dcb6da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java @@ -20,7 +20,7 @@ package org.apache.flink.runtime.operators.chaining; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.Mappable; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -28,15 +28,15 @@ public class ChainedMapDriver extends ChainedDriver { - private Mappable mapper; + private MapFunction mapper; // -------------------------------------------------------------------------------------------- @Override public void setup(AbstractInvokable parent) { @SuppressWarnings("unchecked") - final Mappable mapper = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, Mappable.class); + final MapFunction mapper = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, MapFunction.class); this.mapper = mapper; FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java index 6f6c72ea33e55..ffac15133f262 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -52,7 +52,7 @@ public class SynchronousChainedCombineDriver extends ChainedDriver { private InMemorySorter sorter; - private FlatCombinable combiner; + private FlatCombineFunction combiner; private TypeSerializer serializer; @@ -73,8 +73,8 @@ public void setup(AbstractInvokable parent) { this.parent = parent; @SuppressWarnings("unchecked") - final FlatCombinable combiner = - RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatCombinable.class); + final FlatCombineFunction combiner = + RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatCombineFunction.class); this.combiner = combiner; FunctionUtils.setFunctionRuntimeContext(combiner, getUdfRuntimeContext()); } @@ -186,7 +186,7 @@ private void sortAndCombine() throws Exception { this.comparator); // cache references on the stack - final FlatCombinable stub = this.combiner; + final FlatCombineFunction stub = this.combiner; final Collector output = this.outputCollector; // run stub implementation diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java index cf2e788cabaef..0daf69bc0a8af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -101,7 +101,7 @@ public void close() { } @Override - public final boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) + public final boolean callWithNextKey(FlatJoinFunction matchFunction, Collector collector) throws Exception { if (this.hashJoin.nextRecord()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java index e5e002ed0502b..70e5afbd6ec7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.util.List; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -100,7 +100,7 @@ public void close() { } @Override - public boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) + public boolean callWithNextKey(FlatJoinFunction matchFunction, Collector collector) throws Exception { if (this.hashJoin.nextRecord()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index d87b32df8efb3..da8a11bdf879b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -29,7 +29,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.FlatCombinable; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -72,7 +72,7 @@ public class CombiningUnilateralSortMerger extends UnilateralSortMerger { */ private static final Log LOG = LogFactory.getLog(CombiningUnilateralSortMerger.class); - private final FlatCombinable combineStub; // the user code stub that does the combining + private final FlatCombineFunction combineStub; // the user code stub that does the combining private Configuration udfConfig; @@ -102,7 +102,7 @@ public class CombiningUnilateralSortMerger extends UnilateralSortMerger { * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to * perform the sort. */ - public CombiningUnilateralSortMerger(FlatCombinable combineStub, MemoryManager memoryManager, IOManager ioManager, + public CombiningUnilateralSortMerger(FlatCombineFunction combineStub, MemoryManager memoryManager, IOManager ioManager, MutableObjectIterator input, AbstractInvokable parentTask, TypeSerializerFactory serializerFactory, TypeComparator comparator, double memoryFraction, int maxNumFileHandles, float startSpillingFraction) @@ -134,7 +134,7 @@ public CombiningUnilateralSortMerger(FlatCombinable combineStub, MemoryManage * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to * perform the sort. */ - public CombiningUnilateralSortMerger(FlatCombinable combineStub, MemoryManager memoryManager, IOManager ioManager, + public CombiningUnilateralSortMerger(FlatCombineFunction combineStub, MemoryManager memoryManager, IOManager ioManager, MutableObjectIterator input, AbstractInvokable parentTask, TypeSerializerFactory serializerFactory, TypeComparator comparator, double memoryFraction, int numSortBuffers, int maxNumFileHandles, @@ -255,7 +255,7 @@ else if (element == endMarker()) { // ------------------- Spilling Phase ------------------------ - final FlatCombinable combineStub = CombiningUnilateralSortMerger.this.combineStub; + final FlatCombineFunction combineStub = CombiningUnilateralSortMerger.this.combineStub; // now that we are actually spilling, take the combiner, and open it try { @@ -467,7 +467,7 @@ protected ChannelWithBlockCount mergeChannels(List channe this.memManager.getPageSize()); final WriterCollector collector = new WriterCollector(output, this.serializer); - final FlatCombinable combineStub = CombiningUnilateralSortMerger.this.combineStub; + final FlatCombineFunction combineStub = CombiningUnilateralSortMerger.this.combineStub; // combine and write to disk try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java index 74c3b3121ee4e..2ed75ae900f34 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java @@ -25,7 +25,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -151,7 +151,7 @@ public void abort() { * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey() */ @Override - public boolean callWithNextKey(final FlatJoinable matchFunction, final Collector collector) + public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) throws Exception { if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { @@ -234,7 +234,7 @@ public boolean callWithNextKey(final FlatJoinable matchFunction, fina * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, - final Iterator valsN, final FlatJoinable matchFunction, final Collector collector) + final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) throws Exception { this.copy1 = this.serializer1.copy(val1, this.copy1); @@ -267,7 +267,7 @@ private void crossFirst1withNValues(final T1 val1, final T2 firstValN, * @throws Exception Forwards all exceptions thrown by the stub. */ private void crossSecond1withNValues(T2 val1, T1 firstValN, - Iterator valsN, FlatJoinable matchFunction, Collector collector) + Iterator valsN, FlatJoinFunction matchFunction, Collector collector) throws Exception { this.copy2 = this.serializer2.copy(val1, this.copy2); @@ -297,7 +297,7 @@ private void crossSecond1withNValues(T2 val1, T1 firstValN, */ private void crossMwithNValues(final T1 firstV1, Iterator spillVals, final T2 firstV2, final Iterator blockVals, - final FlatJoinable matchFunction, final Collector collector) + final FlatJoinFunction matchFunction, final Collector collector) throws Exception { // ================================================== diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java index c2febc66c2558..3ed647d8fe0d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java @@ -22,7 +22,7 @@ import java.io.IOException; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.util.Collector; @@ -60,7 +60,7 @@ public interface JoinTaskIterator * @return True, if a next key exists, false if no more keys exist. * @throws Exception Exceptions from the user code are forwarded. */ - boolean callWithNextKey(FlatJoinable matchFunction, Collector collector) throws Exception; + boolean callWithNextKey(FlatJoinFunction matchFunction, Collector collector) throws Exception; /** * Aborts the matching process. This extra abort method is supplied, because a significant time may pass while diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java index f4ad109a88a0c..3894233cc477c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java @@ -23,13 +23,10 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; -import org.apache.flink.runtime.operators.BuildFirstCachedMatchDriver; -import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver; -import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; @@ -43,7 +40,7 @@ import org.junit.Assert; import org.junit.Test; -public class CachedMatchTaskTest extends DriverTestBase> +public class CachedMatchTaskTest extends DriverTestBase> { private static final long HASH_MEM = 6*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java index e48e20f42782f..9018966bca8ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java @@ -23,8 +23,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.java.record.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.runtime.operators.testutils.DriverTestBase; @@ -35,7 +34,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CoGroupTaskExternalITCase extends DriverTestBase> +public class CoGroupTaskExternalITCase extends DriverTestBase> { private static final long SORT_MEM = 3*1024*1024; @@ -85,7 +84,7 @@ public void testExternalSortCoGroupTask() { Assert.assertEquals("Wrong result set size.", expCnt, this.output.getNumberOfRecords()); } - public static final class MockCoGroupStub extends CoGroupFunction { + public static final class MockCoGroupStub extends org.apache.flink.api.java.record.functions.CoGroupFunction { private static final long serialVersionUID = 1L; private final Record res = new Record(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java index 2d8a5e1115247..6a81a0c017ca3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java @@ -24,8 +24,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.CoGroupable; -import org.apache.flink.api.java.record.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.runtime.operators.CoGroupTaskExternalITCase.MockCoGroupStub; @@ -40,7 +39,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CoGroupTaskTest extends DriverTestBase> +public class CoGroupTaskTest extends DriverTestBase> { private static final long SORT_MEM = 3*1024*1024; @@ -401,7 +400,7 @@ public void run() { Assert.assertTrue("Test threw an exception even though it was properly canceled.", success.get()); } - public static class MockFailingCoGroupStub extends CoGroupFunction { + public static class MockFailingCoGroupStub extends org.apache.flink.api.java.record.functions.CoGroupFunction { private static final long serialVersionUID = 1L; private int cnt = 0; @@ -441,7 +440,7 @@ public void coGroup(Iterator records1, } - public static final class MockDelayingCoGroupStub extends CoGroupFunction { + public static final class MockDelayingCoGroupStub extends org.apache.flink.api.java.record.functions.CoGroupFunction { private static final long serialVersionUID = 1L; @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java index 03160752e61d8..d26e107787d9d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java @@ -24,7 +24,7 @@ import junit.framework.Assert; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.runtime.operators.CombineTaskTest.MockCombiningReduceStub; import org.apache.flink.runtime.operators.testutils.DriverTestBase; @@ -35,7 +35,7 @@ import org.junit.Test; -public class CombineTaskExternalITCase extends DriverTestBase> { +public class CombineTaskExternalITCase extends DriverTestBase> { private static final long COMBINE_MEM = 3 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java index b04e597b07454..ed9e39fce2bcf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java @@ -25,7 +25,7 @@ import junit.framework.Assert; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; @@ -41,7 +41,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CombineTaskTest extends DriverTestBase> +public class CombineTaskTest extends DriverTestBase> { private static final long COMBINE_MEM = 3 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java index 27e32f09ffc11..6d0a9fb545735 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java @@ -21,7 +21,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.Crossable; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.runtime.operators.CrossTaskTest.MockCrossStub; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; @@ -29,7 +29,7 @@ import org.junit.Test; -public class CrossTaskExternalITCase extends DriverTestBase> +public class CrossTaskExternalITCase extends DriverTestBase> { private static final long CROSS_MEM = 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java index 1d5df972b2b42..4f1923e2112df 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskTest.java @@ -23,18 +23,16 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.Crossable; -import org.apache.flink.api.java.record.functions.CrossFunction; +import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator; import org.apache.flink.runtime.operators.testutils.DriverTestBase; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.operators.testutils.TaskCancelThread; import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; import org.junit.Test; -public class CrossTaskTest extends DriverTestBase> +public class CrossTaskTest extends DriverTestBase> { private static final long CROSS_MEM = 1024 * 1024; @@ -581,7 +579,7 @@ public void run() { Assert.assertTrue("Exception was thrown despite proper canceling.", success.get()); } - public static final class MockCrossStub extends CrossFunction { + public static final class MockCrossStub extends org.apache.flink.api.java.record.functions.CrossFunction { private static final long serialVersionUID = 1L; @Override @@ -590,7 +588,7 @@ public Record cross(Record record1, Record record2) throws Exception { } } - public static final class MockFailingCrossStub extends CrossFunction { + public static final class MockFailingCrossStub extends org.apache.flink.api.java.record.functions.CrossFunction { private static final long serialVersionUID = 1L; private int cnt = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java index c93d5e1fb93f1..b1937dcb289f5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskExternalITCase.java @@ -21,7 +21,7 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; @@ -33,7 +33,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class MatchTaskExternalITCase extends DriverTestBase> +public class MatchTaskExternalITCase extends DriverTestBase> { private static final long HASH_MEM = 4*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java index 38cdaeee51ba0..9b7c582e1a02a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/MatchTaskTest.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.java.record.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; @@ -40,7 +40,7 @@ import org.junit.Assert; import org.junit.Test; -public class MatchTaskTest extends DriverTestBase> +public class MatchTaskTest extends DriverTestBase> { private static final long HASH_MEM = 6*1024*1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java index 90e6f4f752579..e2a5a0e831275 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java @@ -27,7 +27,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; @@ -41,7 +41,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class ReduceTaskExternalITCase extends DriverTestBase> +public class ReduceTaskExternalITCase extends DriverTestBase> { private static final Log LOG = LogFactory.getLog(ReduceTaskExternalITCase.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java index 8d4bfd4a7a8e7..9cd78b8f78a4d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java @@ -28,7 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; @@ -46,7 +46,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class ReduceTaskTest extends DriverTestBase> +public class ReduceTaskTest extends DriverTestBase> { private static final Log LOG = LogFactory.getLog(ReduceTaskTest.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java index 4becfd8d6c158..d603decdac807 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java @@ -23,8 +23,8 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.common.functions.GroupReducible; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.runtime.operators.AllGroupReduceDriver; @@ -46,8 +46,8 @@ public class AllGroupReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -72,8 +72,8 @@ public void testAllReduceDriverImmutableEmpty() { @Test public void testAllReduceDriverImmutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -112,8 +112,8 @@ public void testAllReduceDriverImmutable() { @Test public void testAllReduceDriverMutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -152,7 +152,7 @@ public void testAllReduceDriverMutable() { // Test UDFs // -------------------------------------------------------------------------------------------- - public static final class ConcatSumReducer extends GroupReduceFunction, Tuple2> { + public static final class ConcatSumReducer extends RichGroupReduceFunction, Tuple2> { @Override public void reduce(Iterator> values, Collector> out) { @@ -169,7 +169,7 @@ public void reduce(Iterator> values, Collector, Tuple2> { + public static final class ConcatSumMutableReducer extends RichGroupReduceFunction, Tuple2> { @Override public void reduce(Iterator> values, Collector> out) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java index ab3166df08878..b124e51159593 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java @@ -22,8 +22,8 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.api.common.functions.Reducible; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.runtime.operators.AllReduceDriver; @@ -44,8 +44,8 @@ public class AllReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -71,8 +71,8 @@ public void testAllReduceDriverImmutableEmpty() { public void testAllReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -103,8 +103,8 @@ public void testAllReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -145,8 +145,8 @@ public void testAllReduceDriverImmutable() { public void testAllReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -176,8 +176,8 @@ public void testAllReduceDriverMutable() { Assert.assertEquals(78, res.f1.getValue()); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); @@ -217,7 +217,7 @@ public void testAllReduceDriverMutable() { // Test UDFs // -------------------------------------------------------------------------------------------- - public static final class ConcatSumFirstReducer extends ReduceFunction> { + public static final class ConcatSumFirstReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -227,7 +227,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumSecondReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -237,7 +237,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumFirstMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -247,7 +247,7 @@ public Tuple2 reduce(Tuple2 value1 } } - public static final class ConcatSumSecondMutableReducer extends ReduceFunction> { + public static final class ConcatSumSecondMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java index c5924ba380059..a325e33a35ad3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java @@ -22,9 +22,9 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -46,8 +46,8 @@ public class GroupReduceDriverTest { @Test public void testAllReduceDriverImmutableEmpty() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -74,8 +74,8 @@ public void testAllReduceDriverImmutableEmpty() { @Test public void testAllReduceDriverImmutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -110,8 +110,8 @@ public void testAllReduceDriverImmutable() { @Test public void testAllReduceDriverMutable() { try { - TestTaskContext, Tuple2>, Tuple2> context = - new TestTaskContext, Tuple2>, Tuple2>(); + TestTaskContext, Tuple2>, Tuple2> context = + new TestTaskContext, Tuple2>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -149,7 +149,7 @@ public void testAllReduceDriverMutable() { // Test UDFs // -------------------------------------------------------------------------------------------- - public static final class ConcatSumReducer extends GroupReduceFunction, Tuple2> { + public static final class ConcatSumReducer extends RichGroupReduceFunction, Tuple2> { @Override public void reduce(Iterator> values, Collector> out) { @@ -166,7 +166,7 @@ public void reduce(Iterator> values, Collector, Tuple2> { + public static final class ConcatSumMutableReducer extends RichGroupReduceFunction, Tuple2> { @Override public void reduce(Iterator> values, Collector> out) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java index 981e7e579d99b..ae9c294a9307b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java @@ -22,9 +22,9 @@ import java.util.Collections; import java.util.List; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -44,8 +44,8 @@ public class ReduceCombineDriverTest { @Test public void testImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -81,8 +81,8 @@ public void testImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -112,8 +112,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceImmutableData(); @@ -153,8 +153,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); @@ -181,8 +181,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(1024 * 1024); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(1024 * 1024); context.getTaskConfig().setRelativeMemoryDriver(0.5); List> data = DriverTestData.createReduceMutableData(); @@ -220,7 +220,7 @@ public void testReduceDriverMutable() { // Test UDFs // -------------------------------------------------------------------------------------------- - public static final class ConcatSumFirstReducer extends ReduceFunction> { + public static final class ConcatSumFirstReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -230,7 +230,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumSecondReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -240,7 +240,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumFirstMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -250,7 +250,7 @@ public Tuple2 reduce(Tuple2 value1 } } - public static final class ConcatSumSecondMutableReducer extends ReduceFunction> { + public static final class ConcatSumSecondMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java index c7ba2a980d7cb..28217b4d1df5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java @@ -21,9 +21,9 @@ import java.util.List; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -43,8 +43,8 @@ public class ReduceDriverTest { @Test public void testReduceDriverImmutableEmpty() { try { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -76,8 +76,8 @@ public void testReduceDriverImmutableEmpty() { public void testReduceDriverImmutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -104,8 +104,8 @@ public void testReduceDriverImmutable() { } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -142,8 +142,8 @@ public void testReduceDriverImmutable() { public void testReduceDriverMutable() { try { { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -169,8 +169,8 @@ public void testReduceDriverMutable() { DriverTestData.compareTupleArrays(expected, res); } { - TestTaskContext>, Tuple2> context = - new TestTaskContext>, Tuple2>(); + TestTaskContext>, Tuple2> context = + new TestTaskContext>, Tuple2>(); List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); @@ -206,7 +206,7 @@ public void testReduceDriverMutable() { // Test UDFs // -------------------------------------------------------------------------------------------- - public static final class ConcatSumFirstReducer extends ReduceFunction> { + public static final class ConcatSumFirstReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -216,7 +216,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumSecondReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -226,7 +226,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> { + public static final class ConcatSumFirstMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { @@ -236,7 +236,7 @@ public Tuple2 reduce(Tuple2 value1 } } - public static final class ConcatSumSecondMutableReducer extends ReduceFunction> { + public static final class ConcatSumSecondMutableReducer extends RichReduceFunction> { @Override public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java index 2d1e5b0a4f2d5..ac8cc782548c0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java @@ -27,7 +27,7 @@ import java.util.Map.Entry; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -412,7 +412,7 @@ public void testBuildFirstWithMixedDataTypes() { collectIntPairData(input1), collectRecordData(input2)); - final FlatJoinable matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final FlatJoinFunction matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); // reset the generators @@ -459,7 +459,7 @@ public void testBuildSecondWithMixedDataTypes() { collectIntPairData(input1), collectRecordData(input2)); - final FlatJoinable matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final FlatJoinFunction matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); // reset the generators @@ -693,7 +693,7 @@ public void join(Record rec1, Record rec2, Collector out) throws Excepti } } - static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinable + static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinFunction { private final Map> toRemoveFrom; diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java index 15cfac365270f..e0a96b3588204 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java @@ -23,9 +23,9 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.JoinFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.runtime.operators.DriverStrategy; @@ -106,7 +106,7 @@ public static DataSet> constructPlan(DataSet, Tuple2, Tuple2> { + public static final class SummingJoin extends RichJoinFunction, Tuple2, Tuple2> { @Override public Tuple2 join(Tuple2 first, Tuple2 second) { @@ -114,7 +114,7 @@ public Tuple2 join(Tuple2 first, Tuple2, Tuple2, Tuple2> { + public static final class SummingJoinProject extends RichJoinFunction, Tuple2, Tuple2> { @Override public Tuple2 join(Tuple3 first, Tuple2 second) { @@ -122,7 +122,7 @@ public Tuple2 join(Tuple3 first, Tuple2, Tuple2> { + public static final class Duplicator extends RichFlatMapFunction, Tuple2> { @Override public void flatMap(Tuple2 value, Collector> out) { @@ -131,7 +131,7 @@ public void flatMap(Tuple2 value, Collector> } } - public static final class Expander extends MapFunction, Tuple3> { + public static final class Expander extends RichMapFunction, Tuple3> { @Override public Tuple3 map(Tuple2 value) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java index e17c089a58cf6..c6d7008e7b6ad 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java @@ -24,7 +24,7 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; @@ -61,7 +61,7 @@ protected void testProgram() throws Exception { } - public static class PickOneAllReduce extends GroupReduceFunction { + public static class PickOneAllReduce extends RichGroupReduceFunction { private Integer bcValue; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java index 31540dc42a8c5..49bdb266f64f0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java @@ -23,9 +23,9 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.java.functions.CoGroupFunction; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichCoGroupFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; import org.apache.flink.api.java.tuple.Tuple2; @@ -90,7 +90,7 @@ protected void testProgram() throws Exception { // The test program // -------------------------------------------------------------------------------------------- - public static final class VertexParser extends MapFunction { + public static final class VertexParser extends RichMapFunction { @Override public Long map(String value) throws Exception { @@ -98,7 +98,7 @@ public Long map(String value) throws Exception { } } - public static final class EdgeParser extends FlatMapFunction> { + public static final class EdgeParser extends RichFlatMapFunction> { @Override public void flatMap(String value, Collector> out) throws Exception { @@ -113,7 +113,7 @@ public void flatMap(String value, Collector> out) throws Exce @ConstantFieldsFirst("0") @ConstantFieldsSecond("0") - public static final class MinIdAndUpdate extends CoGroupFunction, Tuple2, Tuple2> { + public static final class MinIdAndUpdate extends RichCoGroupFunction, Tuple2, Tuple2> { @Override public void coGroup(Iterator> candidates, Iterator> current, Collector> out) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java index be24662bb3e5a..d2669aa927779 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java @@ -23,9 +23,9 @@ import java.util.Iterator; import java.util.List; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.test.util.JavaProgramTestBase; @@ -153,8 +153,7 @@ public Long getKey(Tuple2 vertexWithId) } } - public static final class FindCandidatesJoin extends JoinFunction - , Tuple2, Long> { + public static final class FindCandidatesJoin extends RichJoinFunction, Tuple2, Long> { private static final long serialVersionUID = 1L; @@ -166,7 +165,7 @@ public Long join(Tuple2 vertexWithCompId, } } - public static final class RemoveDuplicatesReduce extends GroupReduceFunction { + public static final class RemoveDuplicatesReduce extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; @@ -176,8 +175,7 @@ public void reduce(Iterator values, Collector out) throws Exception } } - public static final class FindCandidatesDependenciesJoin extends JoinFunction - ,Tuple2> { + public static final class FindCandidatesDependenciesJoin extends RichJoinFunction,Tuple2> { private static final long serialVersionUID = 1L; @@ -187,8 +185,7 @@ public Tuple2 join(Long candidateId, Tuple2 edge) throws } } - public static final class NeighborWithComponentIDJoin extends JoinFunction - , Tuple2, Tuple2> { + public static final class NeighborWithComponentIDJoin extends RichJoinFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; @@ -201,8 +198,7 @@ public Tuple2 join(Tuple2 edge, } } - public static final class MinimumReduce extends GroupReduceFunction - , Tuple2> { + public static final class MinimumReduce extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; final Tuple2 resultVertex = new Tuple2(); @@ -228,8 +224,7 @@ public void reduce(Iterator> values, } } - public static final class MinimumIdFilter extends FlatMapFunction - , Tuple2>, Tuple2> { + public static final class MinimumIdFilter extends RichFlatMapFunction, Tuple2>, Tuple2> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java index 38e70fa39eb64..b97615531b143 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java @@ -28,8 +28,8 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -287,7 +287,7 @@ public boolean isConverged(int iteration, LongValue value) { } @SuppressWarnings("serial") - public static final class SubtractOneMap extends MapFunction { + public static final class SubtractOneMap extends RichMapFunction { private LongSumAggregator aggr; @@ -309,7 +309,7 @@ public Integer map(Integer value) { } @SuppressWarnings("serial") - public static final class SubtractOneMapWithParam extends MapFunction { + public static final class SubtractOneMapWithParam extends RichMapFunction { private LongSumAggregatorWithParameter aggr; @@ -345,7 +345,7 @@ public int getValue() { } @SuppressWarnings("serial") - public static final class TupleMakerMap extends MapFunction> { + public static final class TupleMakerMap extends RichMapFunction> { @Override public Tuple2 map(Integer value) throws Exception { @@ -357,7 +357,7 @@ public Tuple2 map(Integer value) throws Exception { } @SuppressWarnings("serial") - public static final class AggregateMapDelta extends MapFunction, Tuple2> { + public static final class AggregateMapDelta extends RichMapFunction, Tuple2> { private LongSumAggregator aggr; private LongValue previousAggr; @@ -388,8 +388,8 @@ public Tuple2 map(Tuple2 value) { } @SuppressWarnings("serial") - public static final class UpdateFilter extends FlatMapFunction, Tuple2>, - Tuple2> { + public static final class UpdateFilter extends RichFlatMapFunction, Tuple2>, + Tuple2> { private int superstep; @@ -411,7 +411,7 @@ public void flatMap(Tuple2, Tuple2> v } @SuppressWarnings("serial") - public static final class ProjectSecondMapper extends MapFunction, Integer> { + public static final class ProjectSecondMapper extends RichMapFunction, Integer> { @Override public Integer map(Tuple2 value) { @@ -420,7 +420,7 @@ public Integer map(Tuple2 value) { } @SuppressWarnings("serial") - public static final class AggregateMapDeltaWithParam extends MapFunction, Tuple2> { + public static final class AggregateMapDeltaWithParam extends RichMapFunction, Tuple2> { private LongSumAggregatorWithParameter aggr; private LongValue previousAggr; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java index 00428d5e64982..df0ac5e92d1a7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java @@ -24,9 +24,9 @@ import java.util.List; import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; @@ -147,8 +147,7 @@ public static String runProgram(String resultPath) throws Exception { } } - public static final class NeighborWithComponentIDJoin extends JoinFunction - , Tuple2, Tuple2> { + public static final class NeighborWithComponentIDJoin extends RichJoinFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; @@ -161,8 +160,7 @@ public Tuple2 join(Tuple2 vertexWithCompId, } } - public static final class MinimumReduce extends GroupReduceFunction - , Tuple2> { + public static final class MinimumReduce extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; final Tuple2 resultVertex = new Tuple2(); @@ -189,8 +187,7 @@ public void reduce(Iterator> values, } @SuppressWarnings("serial") - public static final class MinimumIdFilter extends FlatMapFunction - , Tuple2>, Tuple2> { + public static final class MinimumIdFilter extends RichFlatMapFunction, Tuple2>, Tuple2> { private static LongSumAggregatorWithParameter aggr; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java index fa1676ff5d7b1..104c3dfecdd97 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java @@ -25,9 +25,9 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.java.functions.FlatMapFunction; -import org.apache.flink.api.java.functions.GroupReduceFunction; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichJoinFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; @@ -139,8 +139,7 @@ public static String runProgram(String resultPath) throws Exception { } } - public static final class NeighborWithComponentIDJoin extends JoinFunction - , Tuple2, Tuple2> { + public static final class NeighborWithComponentIDJoin extends RichJoinFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; @@ -153,8 +152,7 @@ public Tuple2 join(Tuple2 vertexWithCompId, } } - public static final class MinimumReduce extends GroupReduceFunction - , Tuple2> { + public static final class MinimumReduce extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; final Tuple2 resultVertex = new Tuple2(); @@ -181,8 +179,7 @@ public void reduce(Iterator> values, } @SuppressWarnings("serial") - public static final class MinimumIdFilter extends FlatMapFunction - , Tuple2>, Tuple2> { + public static final class MinimumIdFilter extends RichFlatMapFunction, Tuple2>, Tuple2> { private static LongSumAggregator aggr; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index 7f1867f66ddf3..1ec0eb4105bac 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -23,7 +23,7 @@ import java.util.Set; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.CoGroupable; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; @@ -32,7 +32,7 @@ import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator; import org.apache.flink.util.Collector; -public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements CoGroupable { +public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements CoGroupFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java index ad2436fb36071..b44d914bb60cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java @@ -23,7 +23,7 @@ import java.util.Set; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatJoinable; +import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.iterative.nephele.ConfigUtils; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList; @@ -32,7 +32,7 @@ import org.apache.flink.util.Collector; public class CustomCompensatableDotProductMatch extends AbstractRichFunction implements - FlatJoinable + FlatJoinFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java index 6d9e49009db0d..1e08a9f5c6af4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java @@ -21,14 +21,14 @@ import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; import org.apache.flink.util.Collector; -public class CustomRankCombiner extends AbstractRichFunction implements GroupReducible, - FlatCombinable +public class CustomRankCombiner extends AbstractRichFunction implements GroupReduceFunction, + FlatCombineFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java index b914c1c5c1007..3749c1d2e9941 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java @@ -24,8 +24,9 @@ import java.util.Iterator; import java.util.LinkedList; -import org.apache.flink.api.java.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.functions.RichCoGroupFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; @@ -301,7 +302,7 @@ public Integer getKey(CustomType in) { } - public static class Tuple5CoGroup extends CoGroupFunction, Tuple5, Tuple2> { + public static class Tuple5CoGroup implements CoGroupFunction, Tuple5, Tuple2> { private static final long serialVersionUID = 1L; @@ -330,7 +331,7 @@ public void coGroup( } } - public static class CustomTypeCoGroup extends CoGroupFunction { + public static class CustomTypeCoGroup implements CoGroupFunction { private static final long serialVersionUID = 1L; @@ -358,7 +359,7 @@ public void coGroup(Iterator first, } - public static class MixedCoGroup extends CoGroupFunction, CustomType, Tuple3> { + public static class MixedCoGroup implements CoGroupFunction, CustomType, Tuple3> { private static final long serialVersionUID = 1L; @@ -388,7 +389,7 @@ public void coGroup( } - public static class MixedCoGroup2 extends CoGroupFunction, CustomType> { + public static class MixedCoGroup2 implements CoGroupFunction, CustomType> { private static final long serialVersionUID = 1L; @@ -417,7 +418,7 @@ public void coGroup(Iterator first, } - public static class Tuple3ReturnLeft extends CoGroupFunction, Tuple3, Tuple3> { + public static class Tuple3ReturnLeft implements CoGroupFunction, Tuple3, Tuple3> { private static final long serialVersionUID = 1L; @@ -434,7 +435,7 @@ public void coGroup(Iterator> first, } } - public static class Tuple5ReturnRight extends CoGroupFunction, Tuple5, Tuple5> { + public static class Tuple5ReturnRight implements CoGroupFunction, Tuple5, Tuple5> { private static final long serialVersionUID = 1L; @@ -456,7 +457,7 @@ public void coGroup( } - public static class Tuple5CoGroupBC extends CoGroupFunction, Tuple5, Tuple3> { + public static class Tuple5CoGroupBC extends RichCoGroupFunction, Tuple5, Tuple3> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java index 22c3588665f18..304dda2890461 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java @@ -23,8 +23,8 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.common.functions.Crossable; -import org.apache.flink.api.java.functions.CrossFunction; +import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.java.functions.RichCrossFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; @@ -33,7 +33,6 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.JavaProgramTestBase; -import org.apache.flink.util.Collector; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -43,7 +42,7 @@ @RunWith(Parameterized.class) public class CrossITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 12; + private static int NUM_PROGRAMS = 11; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -394,39 +393,6 @@ public static String runProgram(int progId, String resultPath) throws Exception "4,4,Hallo Welt wieHello world\n"; } - case 12: { - /* - * check passing SAM interface instead of rich function - * funcitonality identical to test 1 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = ds - .cross(ds2) - .with(new Crossable, Tuple5, Tuple2>() { - @Override - public Tuple2 cross(Tuple5 first, Tuple5 second) throws Exception { - return new Tuple2(first.f2+second.f2, first.f3+second.f3); - } - }); - - crossDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "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"; - } default: throw new IllegalArgumentException("Invalid program id"); } @@ -435,7 +401,7 @@ public Tuple2 cross(Tuple5, Tuple5, Tuple2> { + public static class Tuple5Cross implements CrossFunction, Tuple5, Tuple2> { private static final long serialVersionUID = 1L; @@ -451,7 +417,7 @@ public Tuple2 cross( } - public static class CustomTypeCross extends CrossFunction { + public static class CustomTypeCross implements CrossFunction { private static final long serialVersionUID = 1L; @@ -464,7 +430,7 @@ public CustomType cross(CustomType first, CustomType second) } - public static class MixedCross extends CrossFunction, CustomType, Tuple3> { + public static class MixedCross implements CrossFunction, CustomType, Tuple3> { private static final long serialVersionUID = 1L; @@ -479,7 +445,7 @@ public Tuple3 cross( } - public static class Tuple3ReturnLeft extends CrossFunction, Tuple5, Tuple3> { + public static class Tuple3ReturnLeft implements CrossFunction, Tuple5, Tuple3> { private static final long serialVersionUID = 1L; @@ -492,7 +458,7 @@ public Tuple3 cross( } } - public static class Tuple5ReturnRight extends CrossFunction, Tuple5, Tuple5> { + public static class Tuple5ReturnRight implements CrossFunction, Tuple5, Tuple5> { private static final long serialVersionUID = 1L; @@ -508,7 +474,7 @@ public Tuple5 cross( } - public static class Tuple5CrossBC extends CrossFunction, Tuple5, Tuple3> { + public static class Tuple5CrossBC extends RichCrossFunction, Tuple5, Tuple3> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java index 203117c40a80b..0c6f3cce08c04 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java @@ -23,7 +23,7 @@ import java.util.LinkedList; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; @@ -164,7 +164,7 @@ public Integer getKey(CustomType in) { return in.myInt; } }) - .map(new MapFunction>() { + .map(new RichMapFunction>() { @Override public Tuple1 map(CustomType value) throws Exception { return new Tuple1(value.myInt); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java index 40c7ab42c2781..6613bc1a63e4c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java @@ -23,8 +23,8 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.common.functions.Filterable; -import org.apache.flink.api.java.functions.FilterFunction; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.java.functions.RichFilterFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -39,7 +39,7 @@ @RunWith(Parameterized.class) public class FilterITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; + private static int NUM_PROGRAMS = 8; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -269,7 +269,7 @@ public boolean filter(CustomType value) throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. - filter(new FilterFunction>() { + filter(new RichFilterFunction>() { private static final long serialVersionUID = 1L; int literal = -1; @@ -307,7 +307,7 @@ public boolean filter(Tuple3 value) throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. - filter(new FilterFunction>() { + filter(new RichFilterFunction>() { private static final long serialVersionUID = 1L; private int broadcastSum = 0; @@ -335,29 +335,6 @@ public boolean filter(Tuple3 value) throws Exception { "15,5,Comment#9\n"; } - case 9: { - /* - * Passing interface instead of function - * Functionality identical to org.apache.flink.test.javaApiOperators.FilterITCase test 3 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = ds. - filter(new Filterable>() { - @Override - public boolean filter(Tuple3 value) throws Exception { - return value.f2.contains("world"); - } - }); - filterDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n"; - } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java index e3cbed434ecd3..a6dd37722c4d4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java @@ -23,8 +23,8 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.common.functions.FlatMappable; -import org.apache.flink.api.java.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.functions.RichFlatMapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -40,7 +40,7 @@ @RunWith(Parameterized.class) public class FlatMapITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 7; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -324,7 +324,7 @@ public void flatMap( Tuple3 value, DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> bcFlatMapDs = ds. - flatMap(new FlatMapFunction, Tuple3>() { + flatMap(new RichFlatMapFunction, Tuple3>() { private static final long serialVersionUID = 1L; private final Tuple3 outTuple = new Tuple3(); @@ -373,45 +373,6 @@ public void flatMap(Tuple3 value, "55,6,Comment#14\n" + "55,6,Comment#15\n"; } - case 8: { - /* - * Pass interface instead of rich function - * Identical to test 3 - * - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> varyingTuplesMapDs = ds. - flatMap(new FlatMappable, Tuple3>() { - @Override - public void flatMap(Tuple3 value, Collector> out) throws Exception { - final int numTuples = value.f0 % 3; - for ( int i = 0; i < numTuples; i++ ) { - out.collect(value); - } - } - }); - - varyingTuplesMapDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "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"; - } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index 638403ddffee4..7376e86e938b3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -24,12 +24,11 @@ import java.util.Iterator; import java.util.LinkedList; -import org.apache.flink.api.common.functions.FlatCombinable; -import org.apache.flink.api.common.functions.GroupReducible; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.java.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; @@ -49,7 +48,7 @@ @RunWith(Parameterized.class) public class GroupReduceITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 14; + private static int NUM_PROGRAMS = 13; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -411,46 +410,6 @@ public Integer getKey(CustomType in) { "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - } - case 14: { - /* - * check correctness of passing interface without combiner - * logic identical to test 9 - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds - .groupBy(1) - .reduceGroup(new GroupReducible, Tuple3>() { - @Override - public void reduce(Iterator> values, Collector> out) throws Exception { - while(values.hasNext()) { - Tuple3 t = values.next(); - - if(t.f0 < 4) { - t.f2 = "Hi!"; - t.f0 += 10; - out.collect(t); - t.f0 += 10; - t.f2 = "Hi again!"; - out.collect(t); - } - } - } - }); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "11,1,Hi!\n" + - "21,1,Hi again!\n" + - "12,2,Hi!\n" + - "22,2,Hi again!\n" + - "13,2,Hi!\n" + - "23,2,Hi again!\n"; - } default: { throw new IllegalArgumentException("Invalid program id"); @@ -460,7 +419,7 @@ public void reduce(Iterator> values, Collector, Tuple2> { + public static class Tuple3GroupReduce implements GroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -482,7 +441,7 @@ public void reduce(Iterator> values, } } - public static class Tuple3SortedGroupReduce extends GroupReduceFunction, Tuple3> { + public static class Tuple3SortedGroupReduce implements GroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; @@ -508,7 +467,7 @@ public void reduce(Iterator> values, } } - public static class Tuple5GroupReduce extends GroupReduceFunction, Tuple5> { + public static class Tuple5GroupReduce implements GroupReduceFunction, Tuple5> { private static final long serialVersionUID = 1L; @Override @@ -532,7 +491,7 @@ public void reduce( } } - public static class CustomTypeGroupReduce extends GroupReduceFunction { + public static class CustomTypeGroupReduce implements GroupReduceFunction { private static final long serialVersionUID = 1L; @@ -559,7 +518,7 @@ public void reduce(Iterator values, } - public static class InputReturningTuple3GroupReduce extends GroupReduceFunction, Tuple3> { + public static class InputReturningTuple3GroupReduce implements GroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; @Override @@ -581,7 +540,7 @@ public void reduce(Iterator> values, } } - public static class AllAddingTuple3GroupReduce extends GroupReduceFunction, Tuple3> { + public static class AllAddingTuple3GroupReduce implements GroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; @Override @@ -601,7 +560,7 @@ public void reduce(Iterator> values, } } - public static class AllAddingCustomTypeGroupReduce extends GroupReduceFunction { + public static class AllAddingCustomTypeGroupReduce implements GroupReduceFunction { private static final long serialVersionUID = 1L; @Override @@ -626,7 +585,7 @@ public void reduce(Iterator values, } } - public static class BCTuple3GroupReduce extends GroupReduceFunction,Tuple3> { + public static class BCTuple3GroupReduce extends RichGroupReduceFunction,Tuple3> { private static final long serialVersionUID = 1L; private String f2Replace = ""; @@ -660,8 +619,8 @@ public void reduce(Iterator> values, } } - @org.apache.flink.api.java.functions.GroupReduceFunction.Combinable - public static class Tuple3GroupReduceWithCombine extends GroupReduceFunction, Tuple2> { + @RichGroupReduceFunction.Combinable + public static class Tuple3GroupReduceWithCombine extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; @Override @@ -697,8 +656,8 @@ public void reduce(Iterator> values, } } - @org.apache.flink.api.java.functions.GroupReduceFunction.Combinable - public static class Tuple3AllGroupReduceWithCombine extends GroupReduceFunction, Tuple2> { + @RichGroupReduceFunction.Combinable + public static class Tuple3AllGroupReduceWithCombine extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; @Override @@ -733,8 +692,8 @@ public void reduce(Iterator> values, Collector { + @RichGroupReduceFunction.Combinable + public static class CustomTypeGroupReduceWithCombine extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; @Override @@ -770,7 +729,7 @@ public void reduce(Iterator values, } } - public static final class IdentityMapper extends MapFunction { + public static final class IdentityMapper extends RichMapFunction { @Override public T map(T value) { return value; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java index e22844cdf9905..a293cbf06a3f9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java @@ -23,10 +23,10 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.common.functions.Joinable; -import org.apache.flink.api.java.functions.FlatJoinFunction; -import org.apache.flink.api.java.functions.JoinFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.functions.RichFlatJoinFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; @@ -461,7 +461,7 @@ public Integer getKey(CustomType value) { } - public static class T3T5FlatJoin extends FlatJoinFunction, Tuple5, Tuple2> { + public static class T3T5FlatJoin implements FlatJoinFunction, Tuple5, Tuple2> { @Override public void join(Tuple3 first, @@ -473,7 +473,7 @@ public void join(Tuple3 first, } - public static class LeftReturningJoin implements Joinable, Tuple5, Tuple3> { + public static class LeftReturningJoin implements JoinFunction, Tuple5, Tuple3> { @Override public Tuple3 join(Tuple3 first, @@ -483,7 +483,7 @@ public Tuple3 join(Tuple3 first, } } - public static class RightReturningJoin extends JoinFunction, Tuple5, Tuple5> { + public static class RightReturningJoin implements JoinFunction, Tuple5, Tuple5> { @Override public Tuple5 join(Tuple3 first, @@ -493,7 +493,7 @@ public Tuple5 join(Tuple3, Tuple5, Tuple3> { + public static class T3T5BCJoin extends RichFlatJoinFunction, Tuple5, Tuple3> { private int broadcast; @@ -525,7 +525,7 @@ public void join(Tuple3 first, Tuple5, CustomType, Tuple2> { + public static class T3CustJoin implements JoinFunction, CustomType, Tuple2> { @Override public Tuple2 join(Tuple3 first, @@ -535,7 +535,7 @@ public Tuple2 join(Tuple3 first, } } - public static class CustT3Join extends JoinFunction, Tuple2> { + public static class CustT3Join implements JoinFunction, Tuple2> { @Override public Tuple2 join(CustomType first, Tuple3 second) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java index 34504fa1e915c..a9b8f331a4c04 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java @@ -25,8 +25,8 @@ import junit.framework.Assert; -import org.apache.flink.api.common.functions.Mappable; -import org.apache.flink.api.java.functions.MapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -41,7 +41,7 @@ @RunWith(Parameterized.class) public class MapITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 10; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -397,7 +397,7 @@ public Tuple3 map(Tuple3 value) DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> bcMapDs = ds. - map(new MapFunction, Tuple3>() { + map(new RichMapFunction, Tuple3>() { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); private Integer f2Replace = 0; @@ -458,7 +458,7 @@ public Tuple3 map(Tuple3 value) final int testValue = 666; conf.setInteger(testKey, testValue); DataSet> bcMapDs = ds. - map(new MapFunction, Tuple3>() { + map(new RichMapFunction, Tuple3>() { private static final long serialVersionUID = 1L; @Override @@ -480,39 +480,6 @@ public Tuple3 map(Tuple3 value) { + "2,2,Hello\n" + "3,2,Hello world"; } - case 10: { - /* - * Test passing interface instead of rich function - * Identical to test 4 - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet typeConversionMapDs = ds. - map(new Mappable, String>() { - @Override - public String map(Tuple3 record) throws Exception { - return record.getField(2); - } - }); - - typeConversionMapDs.writeAsText(resultPath); - env.execute(); - - // return expected result - return "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"; - - } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java index 76040961ead57..a296a099aa23a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java @@ -23,9 +23,9 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.common.functions.Reducible; +import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.configuration.Configuration; @@ -41,7 +41,7 @@ @RunWith(Parameterized.class) public class ReduceITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; + private static int NUM_PROGRAMS = 8; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -271,42 +271,6 @@ public Integer getKey(CustomType in) { "65,5,Hi again!\n" + "111,6,Hi again!\n"; } - case 9: { - /* - * Passing interface instead of function - * Functionality identical to org.apache.flink.test.javaApiOperators.ReduceITCase test 2 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = ds - .groupBy(4, 0) - .reduce(new Reducible>() { - @Override - public Tuple5 reduce(Tuple5 in1, Tuple5 in2) throws Exception { - Tuple5 out = new Tuple5(); - out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4); - return out; - } - }); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - // return expected result - return "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"; - - } default: throw new IllegalArgumentException("Invalid program id"); } @@ -315,7 +279,7 @@ public Tuple5 reduce(Tuple5> { + public static class Tuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); private final String f2Replace; @@ -343,7 +307,7 @@ public Tuple3 reduce( } } - public static class Tuple5Reduce extends ReduceFunction> { + public static class Tuple5Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple5 out = new Tuple5(); @@ -358,7 +322,7 @@ public Tuple5 reduce( } } - public static class CustomTypeReduce extends ReduceFunction { + public static class CustomTypeReduce implements ReduceFunction { private static final long serialVersionUID = 1L; private final CustomType out = new CustomType(); @@ -373,7 +337,7 @@ public CustomType reduce(CustomType in1, CustomType in2) } } - public static class InputReturningTuple3Reduce extends ReduceFunction> { + public static class InputReturningTuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; @Override @@ -387,7 +351,7 @@ public Tuple3 reduce( } } - public static class AllAddingTuple3Reduce extends ReduceFunction> { + public static class AllAddingTuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); @@ -401,7 +365,7 @@ public Tuple3 reduce( } } - public static class AllAddingCustomTypeReduce extends ReduceFunction { + public static class AllAddingCustomTypeReduce implements ReduceFunction { private static final long serialVersionUID = 1L; private final CustomType out = new CustomType(); @@ -416,7 +380,7 @@ public CustomType reduce(CustomType in1, CustomType in2) } } - public static class BCTuple3Reduce extends ReduceFunction> { + public static class BCTuple3Reduce extends RichReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); private String f2Replace = ""; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java index 09191cc59e8b5..a636ba455745e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java @@ -23,7 +23,7 @@ import java.util.Collection; import java.util.LinkedList; -import org.apache.flink.api.java.functions.FilterFunction; +import org.apache.flink.api.java.functions.RichFilterFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -145,7 +145,7 @@ public static String runProgram(int progId, String resultPath) throws Exception // Don't know how to make an empty result in an other way than filtering it DataSet> empty = CollectionDataSets.get3TupleDataSet(env). - filter(new FilterFunction>() { + filter(new RichFilterFunction>() { private static final long serialVersionUID = 1L; @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java index 97367f768d8d6..7149cd39a3b3b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java @@ -24,8 +24,8 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; -import org.apache.flink.api.java.functions.MapFunction; -import org.apache.flink.api.java.functions.ReduceFunction; +import org.apache.flink.api.java.functions.RichMapFunction; +import org.apache.flink.api.java.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; @@ -170,7 +170,7 @@ public String toString() { // ************************************************************************* /** Converts a Tuple2 into a Point. */ - public static final class TuplePointConverter extends MapFunction, Point> { + public static final class TuplePointConverter extends RichMapFunction, Point> { @Override public Point map(Tuple2 t) throws Exception { @@ -179,7 +179,7 @@ public Point map(Tuple2 t) throws Exception { } /** Converts a Tuple3 into a Centroid. */ - public static final class TupleCentroidConverter extends MapFunction, Centroid> { + public static final class TupleCentroidConverter extends RichMapFunction, Centroid> { @Override public Centroid map(Tuple3 t) throws Exception { @@ -188,7 +188,7 @@ public Centroid map(Tuple3 t) throws Exception { } /** Determines the closest cluster center for a data point. */ - public static final class SelectNearestCenter extends MapFunction> { + public static final class SelectNearestCenter extends RichMapFunction> { private Collection centroids; /** Reads the centroid values from a broadcast variable into a collection. */ @@ -236,7 +236,7 @@ public DummyTuple3IntPointLong() {} } /** Appends a count variable to the tuple. */ - public static final class CountAppender extends MapFunction, DummyTuple3IntPointLong> { + public static final class CountAppender extends RichMapFunction, DummyTuple3IntPointLong> { @Override public DummyTuple3IntPointLong map(Tuple2 t) { @@ -245,7 +245,7 @@ public DummyTuple3IntPointLong map(Tuple2 t) { } /** Sums and counts point coordinates. */ - public static final class CentroidAccumulator extends ReduceFunction { + public static final class CentroidAccumulator extends RichReduceFunction { @Override public DummyTuple3IntPointLong reduce(DummyTuple3IntPointLong val1, DummyTuple3IntPointLong val2) { @@ -254,7 +254,7 @@ public DummyTuple3IntPointLong reduce(DummyTuple3IntPointLong val1, DummyTuple3I } /** Computes new centroid from coordinate sum and count of points. */ - public static final class CentroidAverager extends MapFunction { + public static final class CentroidAverager extends RichMapFunction { @Override public Centroid map(DummyTuple3IntPointLong value) {