You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by rm...@apache.org on 2014/08/09 14:39:51 UTC

[20/39] [FLINK-701] Refactor Java API to use SAM interfaces. Introduce RichFunction stubs for all UDFs.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..a4e1248
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java
@@ -0,0 +1,67 @@
+/**
+ * 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.CrossFunction;
+
+
+/**
+ * The abstract base class for Cross functions. Cross functions build a Cartesian produce of their inputs
+ * and call the function or each pair of elements.
+ * They are a means of convenience and can be used to directly produce manipulate the
+ * pair of elements, instead of having the operator build 2-tuples, and then using a
+ * MapFunction over those 2-tuples.
+ * <p>
+ * The basic syntax for using Cross on two data sets is as follows:
+ * <pre><blockquote>
+ * DataSet<X> set1 = ...;
+ * DataSet<Y> set2 = ...;
+ * 
+ * set1.cross(set2).with(new MyCrossFunction());
+ * </blockquote></pre>
+ * <p>
+ * {@code set1} is here considered the first input, {@code set2} the second input.
+ * <p>
+ * All functions need to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichCrossFunction<IN1, IN2, OUT> extends AbstractRichFunction implements CrossFunction<IN1, IN2, OUT> {
+	
+	private static final long serialVersionUID = 1L;
+	
+
+	/**
+	 * The core method of the cross operation. The method will be invoked for each pair of elements
+	 * in the Cartesian product.
+	 * 
+	 * @param first The element from the first input.
+	 * @param second The element from the second input.
+	 * @return The result element.
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public abstract OUT cross(IN1 first, IN2 second) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e3baa74
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java
@@ -0,0 +1,57 @@
+/**
+ * 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.FilterFunction;
+
+/**
+ * The abstract base class for Filter functions. A filter function take elements and evaluates a
+ * predicate on them to decide whether to keep the element, or to discard it.
+ * <p>
+ * The basic syntax for using a FilterFunction is as follows:
+ * <pre><blockquote>
+ * DataSet<X> input = ...;
+ * 
+ * DataSet<X> result = input.filter(new MyFilterFunction());
+ * </blockquote></pre>
+ * <p>
+ * Like all functions, the FilterFunction needs to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <T> The type of the filtered elements.
+ */
+public abstract class RichFilterFunction<T> extends AbstractRichFunction implements FilterFunction<T> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	/**
+	 * The core method of the FilterFunction. The method is called for each element in the input,
+	 * and determines whether the element should be kept or filtered out. If the method returns true,
+	 * the element passes the filter and is kept, if the method returns false, the element is
+	 * filtered out.
+	 * 
+	 * @param value The input value to be filtered.
+	 * @return Flag to indicate whether to keep the value (true) or to discard it (false).
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public abstract boolean filter(T value) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8c326c6
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.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.FlatCombineFunction;
+import org.apache.flink.util.Collector;
+
+import java.util.Iterator;
+
+public abstract class RichFlatCombineFunction<T> extends AbstractRichFunction implements FlatCombineFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract void combine(Iterator<T> values, Collector<T> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..15b4539
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.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.FlatJoinFunction;
+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.
+ * <p>
+ * Per the semantics of an inner join, the function is 
+ * <p>
+ * The basic syntax for using Join on two data sets is as follows:
+ * <pre><blockquote>
+ * DataSet<X> set1 = ...;
+ * DataSet<Y> set2 = ...;
+ * 
+ * set1.join(set2).where(<key-definition>).equalTo(<key-definition>).with(new MyJoinFunction());
+ * </blockquote></pre>
+ * <p>
+ * {@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.
+ * <p>
+ * 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.
+ * <P>
+ * Note: You can use a {@link RichCoGroupFunction} to perform an outer join.
+ * <p>
+ * All functions need to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichFlatJoinFunction<IN1, IN2, OUT> extends AbstractRichFunction implements FlatJoinFunction<IN1, IN2, OUT> {
+
+	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> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2293b5e
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.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.java.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * 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 RichMapFunction}.
+ * <p>
+ * The basic syntax for using a FlatMapFunction is as follows:
+ * <pre><blockquote>
+ * DataSet<X> input = ...;
+ * 
+ * DataSet<Y> result = input.flatMap(new MyFlatMapFunction());
+ * </blockquote></pre>
+ * <p>
+ * Like all functions, the FlatMapFunction needs to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <IN> Type of the input elements.
+ * @param <OUT> Type of the returned elements.
+ */
+public abstract class RichFlatMapFunction<IN, OUT> extends AbstractRichFunction implements FlatMapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * The core method of the FlatMapFunction. 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.
+	 */
+	@Override
+	public abstract void flatMap(IN value, Collector<OUT> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..eb75f53
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java
@@ -0,0 +1,114 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.util.Iterator;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatCombineFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * The abstract base class for group reduce functions. Group reduce functions process groups of elements.
+ * They may aggregate them to a single value, or produce multiple result values for each group.
+ * <p>
+ * For a reduce functions that works incrementally by combining always two elements, see 
+ * {@link RichReduceFunction}, called via {@link org.apache.flink.api.java.DataSet#reduce(RichReduceFunction)}.
+ * <p>
+ * The basic syntax for using a grouped GroupReduceFunction is as follows:
+ * <pre><blockquote>
+ * DataSet<X> input = ...;
+ * 
+ * DataSet<X> result = input.groupBy(<key-definition>).reduceGroup(new MyGroupReduceFunction());
+ * </blockquote></pre>
+ * <p>
+ * GroupReduceFunctions may be "combinable", in which case they can pre-reduce partial groups in order to
+ * reduce the data volume early. See the {@link #combine(Iterator, Collector)} function for details.
+ * <p>
+ * Like all functions, the GroupReduceFunction needs to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <IN> Type of the elements that this function processes.
+ * @param <OUT> The type of the elements returned by the user-defined function.
+ */
+public abstract class RichGroupReduceFunction<IN, OUT> extends AbstractRichFunction implements GroupReduceFunction<IN, OUT>, FlatCombineFunction<IN> {
+	
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Core method of the reduce function. It is called one per group of elements. If the reducer
+	 * is not grouped, than the entire data set is considered one group.
+	 * 
+	 * @param values The iterator returning the group of values to be reduced.
+	 * @param out The collector to emit the returned values.
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public abstract void reduce(Iterator<IN> values, Collector<OUT> out) throws Exception;
+	
+	/**
+	 * The combine methods pre-reduces elements. It may be called on subsets of the data
+	 * before the actual reduce function. This is often helpful to lower data volume prior
+	 * to reorganizing the data in an expensive way, as might be required for the final
+	 * reduce function.
+	 * <p>
+	 * This method is only ever invoked when the subclass of {@link RichGroupReduceFunction}
+	 * adds the {@link Combinable} annotation, or if the <i>combinable</i> flag is set when defining
+	 * the <i>reduceGroup<i> operation via
+	 * {@link org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean)}.
+	 * <p>
+	 * 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
+	 * {@link #reduce(Iterator, Collector)} method. If the behavior in the pre-reducing is different
+	 * from the final reduce function (for example because the reduce function changes the data type),
+	 * this method must be overwritten, or the execution will fail.
+	 * 
+	 * @param values The iterator returning the group of values to be reduced.
+	 * @param out The collector to emit the returned values.
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public void combine(Iterator<IN> values, Collector<IN> out) throws Exception {
+		@SuppressWarnings("unchecked")
+		Collector<OUT> c = (Collector<OUT>) out;
+		reduce(values, c);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * 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.
+	 * <p>
+	 * Marking combinable functions as such is in general beneficial for performance.
+	 */
+	@Retention(RetentionPolicy.RUNTIME)
+	@Target(ElementType.TYPE)
+	public static @interface Combinable {};
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7eaf44c
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.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.java.functions;
+
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+
+public abstract class RichJoinFunction<IN1,IN2,OUT> extends AbstractRichFunction implements JoinFunction<IN1,IN2,OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract OUT join(IN1 first, IN2 second) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..54de7d4
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.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.java.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+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 RichFlatMapFunction}.
+ * <p>
+ * The basic syntax for using a MapFunction is as follows:
+ * <pre><blockquote>
+ * DataSet<X> input = ...;
+ * 
+ * DataSet<Y> result = input.map(new MyMapFunction());
+ * </blockquote></pre>
+ * <p>
+ * Like all functions, the MapFunction needs to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <IN> Type of the input elements.
+ * @param <OUT> Type of the returned elements.
+ */
+public abstract class RichMapFunction<IN, OUT> extends AbstractRichFunction implements MapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * The core method of the MapFunction. Takes an element from the input data set and transforms
+	 * it into another element.
+	 * 
+	 * @param value The input value.
+	 * @return The value produced by the map function from the input value.
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public abstract OUT map(IN value) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..35cb392
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java
@@ -0,0 +1,63 @@
+/**
+ * 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.ReduceFunction;
+
+/**
+ * The abstract base class for Reduce functions. Reduce functions combine groups of elements to
+ * a single value, by taking always two elements and combining them into one. Reduce functions
+ * may be used on entire data sets, or on grouped data sets. In the latter case, each group is reduced
+ * individually.
+ * <p>
+ * For a reduce functions that work on an entire group at the same time (such as the 
+ * 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.
+ * <p>
+ * The basic syntax for using a grouped ReduceFunction is as follows:
+ * <pre><blockquote>
+ * DataSet<X> input = ...;
+ * 
+ * DataSet<X> result = input.groupBy(<key-definition>).reduce(new MyReduceFunction());
+ * </blockquote></pre>
+ * <p>
+ * Like all functions, the ReduceFunction needs to be serializable, as defined in {@link java.io.Serializable}.
+ * 
+ * @param <T> Type of the elements that this function processes.
+ */
+public abstract class RichReduceFunction<T> extends AbstractRichFunction implements ReduceFunction<T> {
+	
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * The core method of the ReduceFunction, 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.
+	 */
+	public abstract T reduce(T value1, T value2) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java
----------------------------------------------------------------------
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 0000000..ccc4685
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/UnsupportedLambdaExpressionException.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.functions;
+
+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 supported only in filter and reduce user-defined functions.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
----------------------------------------------------------------------
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 ca6ed94..80a5fa0 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 java.util.List;
 
 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.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.common.operators.base.GroupReduceOperatorBase;
 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 class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 
 	@SuppressWarnings("unchecked")
 	@Override
-	protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase<IN, IN, GenericGroupReduce<IN, IN>> translateToDataFlow(Operator<IN> input) {
+	protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> translateToDataFlow(Operator<IN> input) {
 		
 		// sanity check
 		if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) {
@@ -174,7 +174,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 		
 		
 		@SuppressWarnings("rawtypes")
-		GroupReduceFunction<IN, IN> function = new AggregatingUdf(aggFunctions, fields);
+		RichGroupReduceFunction<IN, IN> function = new AggregatingUdf(aggFunctions, fields);
 		
 		
 		String name = getName() != null ? getName() : genName.toString();
@@ -183,8 +183,8 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 		if (this.grouping == null) {
 			// non grouped aggregation
 			UnaryOperatorInformation<IN, IN> operatorInfo = new UnaryOperatorInformation<IN, IN>(getInputType(), getResultType());
-			GroupReduceOperatorBase<IN, IN, GenericGroupReduce<IN, IN>> po =
-					new GroupReduceOperatorBase<IN, IN, GenericGroupReduce<IN, IN>>(function, operatorInfo, new int[0], name);
+			GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> po =
+					new GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>>(function, operatorInfo, new int[0], name);
 			
 			po.setCombinable(true);
 			
@@ -200,8 +200,8 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 			// grouped aggregation
 			int[] logicalKeyPositions = this.grouping.getKeys().computeLogicalKeyPositions();
 			UnaryOperatorInformation<IN, IN> operatorInfo = new UnaryOperatorInformation<IN, IN>(getInputType(), getResultType());
-			GroupReduceOperatorBase<IN, IN, GenericGroupReduce<IN, IN>> po =
-					new GroupReduceOperatorBase<IN, IN, GenericGroupReduce<IN, IN>>(function, operatorInfo, logicalKeyPositions, name);
+			GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> po =
+					new GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>>(function, operatorInfo, logicalKeyPositions, name);
 			
 			po.setCombinable(true);
 			
@@ -245,7 +245,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 	// --------------------------------------------------------------------------------------------
 	
 	@Combinable
-	public static final class AggregatingUdf<T extends Tuple> extends GroupReduceFunction<T, T> {
+	public static final class AggregatingUdf<T extends Tuple> extends RichGroupReduceFunction<T, T> {
 		private static final long serialVersionUID = 1L;
 		
 		private final int[] fieldPositions;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
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 8748556..89c3334 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 @@ package org.apache.flink.api.java.operators;
 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.GenericMap;
+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;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
@@ -30,8 +31,8 @@ import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
 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;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
 import org.apache.flink.api.java.operators.translation.PlanUnwrappingCoGroupOperator;
@@ -123,8 +124,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions();
 			int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions();
 			
-			CoGroupOperatorBase<I1, I2, OUT, GenericCoGrouper<I1, I2, OUT>> po =
-					new CoGroupOperatorBase<I1, I2, OUT, GenericCoGrouper<I1, I2, OUT>>(
+			CoGroupOperatorBase<I1, I2, OUT, CoGroupFunction<I1, I2, OUT>> po =
+					new CoGroupOperatorBase<I1, I2, OUT, CoGroupFunction<I1, I2, OUT>>(
 							function, new BinaryOperatorInformation<I1, I2, OUT>(getInput1Type(), getInput2Type(), getResultType()),
 							logicalKeyPositions1, logicalKeyPositions2, name);
 			
@@ -199,10 +200,10 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		final KeyExtractingMapper<I1, K> extractor1 = new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
 		final KeyExtractingMapper<I2, K> extractor2 = new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
 		
-		final MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>> keyMapper1 =
-				new MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
-		final MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>> keyMapper2 =
-				new MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
+		final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
+				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
+		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
+				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
 		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
 
 		cogroup.setFirstInput(keyMapper1);
@@ -236,10 +237,10 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		final TupleKeyExtractingMapper<I1, K> extractor1 = new TupleKeyExtractingMapper<I1, K>(logicalKeyPositions1[0]);
 		final KeyExtractingMapper<I2, K> extractor2 = new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
 
-		final MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>> keyMapper1 =
-				new MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
-		final MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>> keyMapper2 =
-				new MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
+		final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
+				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
+		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
+				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
 		
 		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
 
@@ -274,10 +275,10 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		final KeyExtractingMapper<I1, K> extractor1 = new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
 		final TupleKeyExtractingMapper<I2, K> extractor2 = new TupleKeyExtractingMapper<I2, K>(logicalKeyPositions2[0]);
 
-		final MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>> keyMapper1 =
-				new MapOperatorBase<I1, Tuple2<K, I1>, GenericMap<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
-		final MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>> keyMapper2 =
-				new MapOperatorBase<I2, Tuple2<K, I2>, GenericMap<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
+		final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
+				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
+		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
+				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
 		
 		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
 
@@ -407,7 +408,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			 * @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];
@@ -423,7 +424,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			 * @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];
@@ -439,7 +440,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			 * 
 			 * @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 <K> CoGroupOperatorWithoutFunction equalTo(KeySelector<I2, K> keyExtractor) {
 				return createCoGroupOperator(new Keys.SelectorFunctionKeys<I2, K>(keyExtractor, input2.getType()));
@@ -447,8 +448,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 
 			/**
 			 * Intermediate step of a CoGroup transformation. <br/>
-			 * 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<I2> keys2) {
@@ -501,19 +502,22 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				}
 
 				/**
-				 * Finalizes a CoGroup transformation by applying a {@link CoGroupFunction} to groups of elements with identical keys.<br/>
+				 * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.java.functions.RichCoGroupFunction} to groups of elements with identical keys.<br/>
 				 * 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 <R> CoGroupOperator<I1, I2, R> with(CoGroupFunction<I1, I2, R> function) {
 					if (function == null) {
 						throw new NullPointerException("CoGroup function must not be null.");
 					}
+					if (FunctionUtils.isSerializedLambdaFunction(function)) {
+						throw new UnsupportedLambdaExpressionException();
+					}
 					TypeInformation<R> returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType());
 					return new CoGroupOperator<I1, I2, R>(input1, input2, keys1, keys2, function, returnType);
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
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 036d292..d1e99d6 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 @@ package org.apache.flink.api.java.operators;
 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.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;
 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;
@@ -71,12 +72,12 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 	}
 
 	@Override
-	protected org.apache.flink.api.common.operators.base.CrossOperatorBase<I1, I2, OUT, GenericCrosser<I1,I2,OUT>> translateToDataFlow(Operator<I1> input1, Operator<I2> input2) {
+	protected org.apache.flink.api.common.operators.base.CrossOperatorBase<I1, I2, OUT, CrossFunction<I1,I2,OUT>> translateToDataFlow(Operator<I1> input1, Operator<I2> input2) {
 		
 		String name = getName() != null ? getName() : function.getClass().getName();
 		// create operator
-		CrossOperatorBase<I1, I2, OUT, GenericCrosser<I1, I2, OUT>> po =
-				new CrossOperatorBase<I1, I2, OUT, GenericCrosser<I1, I2, OUT>>(function, new BinaryOperatorInformation<I1, I2, OUT>(getInput1Type(), getInput2Type(), getResultType()), name);
+		CrossOperatorBase<I1, I2, OUT, CrossFunction<I1, I2, OUT>> po =
+				new CrossOperatorBase<I1, I2, OUT, CrossFunction<I1, I2, OUT>>(function, new BinaryOperatorInformation<I1, I2, OUT>(getInput1Type(), getInput2Type(), getResultType()), name);
 		// set inputs
 		po.setFirstInput(input1);
 		po.setSecondInput(input2);
@@ -133,6 +134,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if (function == null) {
 				throw new NullPointerException("Cross function must not be null.");
 			}
+			if (FunctionUtils.isSerializedLambdaFunction(function)) {
+				throw new UnsupportedLambdaExpressionException();
+			}
 			TypeInformation<R> returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType());
 			return new CrossOperator<I1, I2, R>(input1, input2, function, returnType);
 		}
@@ -220,7 +224,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 	}
 
-	public static final class ProjectCrossFunction<T1, T2, R extends Tuple> extends CrossFunction<T1, T2, R> {
+	public static final class ProjectCrossFunction<T1, T2, R extends Tuple> implements CrossFunction<T1, T2, R> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -1398,7 +1402,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 	//  default join functions
 	// --------------------------------------------------------------------------------------------
 
-	public static final class DefaultCrossFunction<T1, T2> extends CrossFunction<T1, T2, Tuple2<T1, T2>> {
+	public static final class DefaultCrossFunction<T1, T2> implements CrossFunction<T1, T2, Tuple2<T1, T2>> {
 
 		private static final long serialVersionUID = 1L;
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
----------------------------------------------------------------------
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 cb7db06..591551f 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,13 +21,14 @@ package org.apache.flink.api.java.operators;
 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.GenericMap;
+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;
@@ -78,15 +79,17 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 	@Override
 	protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase<?, T, ?> translateToDataFlow(Operator<T> input) {
 		
-		GroupReduceFunction<T, T> function = new DistinctFunction<T>();
+		final RichGroupReduceFunction<T, T> function = new DistinctFunction<T>();
+		final FlatCombineFunction<T> combineFunction = new DistinctCombiner<T>();
+
 		String name = function.getClass().getName();
 		
 		if (keys instanceof Keys.FieldPositionKeys) {
 
 			int[] logicalKeyPositions = keys.computeLogicalKeyPositions();
 			UnaryOperatorInformation<T, T> operatorInfo = new UnaryOperatorInformation<T, T>(getInputType(), getResultType());
-			GroupReduceOperatorBase<T, T, GenericGroupReduce<T, T>> po =
-					new GroupReduceOperatorBase<T, T, GenericGroupReduce<T, T>>(function, operatorInfo, logicalKeyPositions, name);
+			GroupReduceOperatorBase<T, T, GroupReduceFunction<T, T>> po =
+					new GroupReduceOperatorBase<T, T, GroupReduceFunction<T, T>>(function, operatorInfo, logicalKeyPositions, name);
 
 			po.setCombinable(true);
 			po.setInput(input);
@@ -98,9 +101,10 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 		
 			@SuppressWarnings("unchecked")
 			Keys.SelectorFunctionKeys<T, ?> selectorKeys = (Keys.SelectorFunctionKeys<T, ?>) keys;
-			
+
+
 			PlanUnwrappingReduceGroupOperator<T, T, ?> 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 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 	// --------------------------------------------------------------------------------------------
 	
 	private static <IN, OUT, K> PlanUnwrappingReduceGroupOperator<IN, OUT, K> translateSelectorFunctionDistinct(
-			Keys.SelectorFunctionKeys<IN, ?> rawKeys, GroupReduceFunction<IN, OUT> function,
+			Keys.SelectorFunctionKeys<IN, ?> rawKeys, RichGroupReduceFunction<IN, OUT> function, FlatCombineFunction<IN> combineFunction,
 			TypeInformation<IN> inputType, TypeInformation<OUT> outputType, String name, Operator<IN> input,
 			boolean combinable)
 	{
@@ -124,10 +128,12 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 		TypeInformation<Tuple2<K, IN>> typeInfoWithKey = new TupleTypeInfo<Tuple2<K, IN>>(keys.getKeyType(), inputType);
 		
 		KeyExtractingMapper<IN, K> extractor = new KeyExtractingMapper<IN, K>(keys.getKeyExtractor());
+
+
+		PlanUnwrappingReduceGroupOperator<IN, OUT, K> reducer =
+				new PlanUnwrappingReduceGroupOperator<IN, OUT, K>(function, keys, name, outputType, typeInfoWithKey, combinable);
 		
-		PlanUnwrappingReduceGroupOperator<IN, OUT, K> reducer = new PlanUnwrappingReduceGroupOperator<IN, OUT, K>(function, keys, name, outputType, typeInfoWithKey, combinable);
-		
-		MapOperatorBase<IN, Tuple2<K, IN>, GenericMap<IN, Tuple2<K, IN>>> mapper = new MapOperatorBase<IN, Tuple2<K, IN>, GenericMap<IN, Tuple2<K, IN>>>(extractor, new UnaryOperatorInformation<IN, Tuple2<K, IN>>(inputType, typeInfoWithKey), "Key Extractor");
+		MapOperatorBase<IN, Tuple2<K, IN>, MapFunction<IN, Tuple2<K, IN>>> mapper = new MapOperatorBase<IN, Tuple2<K, IN>, MapFunction<IN, Tuple2<K, IN>>>(extractor, new UnaryOperatorInformation<IN, Tuple2<K, IN>>(inputType, typeInfoWithKey), "Key Extractor");
 
 		reducer.setInput(mapper);
 		mapper.setInput(input);
@@ -138,7 +144,7 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 		return reducer;
 	}
 	
-	public static final class DistinctFunction<T> extends GroupReduceFunction<T, T> {
+	public static final class DistinctFunction<T> extends RichGroupReduceFunction<T, T> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -148,4 +154,16 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 			out.collect(values.next());
 		}
 	}
+
+	public static final class DistinctCombiner<T> implements FlatCombineFunction<T> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterator<T> values, Collector<T> out)
+				throws Exception {
+			out.collect(values.next());
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
----------------------------------------------------------------------
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 ed4a786..1c03ccd 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,9 +18,9 @@
 
 package org.apache.flink.api.java.operators;
 
-import org.apache.flink.api.common.functions.GenericFlatMap;
+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.functions.FilterFunction;
 import org.apache.flink.api.java.operators.translation.PlanFilterOperator;
 
 import org.apache.flink.api.java.DataSet;
@@ -44,7 +44,7 @@ public class FilterOperator<T> extends SingleInputUdfOperator<T, T, FilterOperat
 	}
 	
 	@Override
-	protected org.apache.flink.api.common.operators.base.FilterOperatorBase<T, GenericFlatMap<T,T>> translateToDataFlow(Operator<T> input) {
+	protected org.apache.flink.api.common.operators.base.FilterOperatorBase<T, FlatMapFunction<T,T>> translateToDataFlow(Operator<T> input) {
 		
 		String name = getName() != null ? getName() : function.getClass().getName();
 		// create operator

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
----------------------------------------------------------------------
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 14c0819..8e531d4 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.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;
-import org.apache.flink.api.java.functions.FlatMapFunction;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 import org.apache.flink.api.java.DataSet;
@@ -47,11 +46,11 @@ public class FlatMapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, Fl
 	}
 
 	@Override
-	protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase<IN, OUT, GenericFlatMap<IN,OUT>> translateToDataFlow(Operator<IN> input) {
+	protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase<IN, OUT, FlatMapFunction<IN,OUT>> translateToDataFlow(Operator<IN> input) {
 		
 		String name = getName() != null ? getName() : function.getClass().getName();
 		// create operator
-		FlatMapOperatorBase<IN, OUT, GenericFlatMap<IN, OUT>> po = new FlatMapOperatorBase<IN, OUT, GenericFlatMap<IN, OUT>>(function, new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType()), name);
+		FlatMapOperatorBase<IN, OUT, FlatMapFunction<IN, OUT>> po = new FlatMapOperatorBase<IN, OUT, FlatMapFunction<IN, OUT>>(function, new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType()), name);
 		// set input
 		po.setInput(input);
 		// set dop

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7ab0b11
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
@@ -0,0 +1,224 @@
+/**
+ * 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.operators;
+
+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.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;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.types.TypeInformation;
+
+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.
+ * 
+ * @param <IN> The type of the data set consumed by the operator.
+ * @param <OUT> The type of the data set created by the operator.
+ */
+public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, GroupReduceOperator<IN, OUT>> {
+
+	private final GroupReduceFunction<IN, OUT> function;
+
+	private final Grouping<IN> 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 GroupReduceOperator(DataSet<IN> input, GroupReduceFunction<IN, OUT> function) {
+		super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType()));
+		
+		this.function = function;
+		this.grouper = null;
+
+		checkCombinability();
+	}
+	
+	/**
+	 * Constructor for a grouped reduce.
+	 * 
+	 * @param input The grouped input to be processed group-wise by the groupReduce function.
+	 * @param function The user-defined GroupReduce function.
+	 */
+	public GroupReduceOperator(Grouping<IN> input, GroupReduceFunction<IN, OUT> 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 FlatCombineFunction &&
+				function.getClass().getAnnotation(RichGroupReduceFunction.Combinable.class) != null) {
+			this.combinable = true;
+		}
+	}
+
+	
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean isCombinable() {
+		return combinable;
+	}
+	
+	public void setCombinable(boolean combinable) {
+		// sanity check that the function is a subclass of the combine interface
+		if (combinable && !(function instanceof FlatCombineFunction)) {
+			throw new IllegalArgumentException("The function does not implement the combine interface.");
+		}
+		
+		this.combinable = combinable;
+	}
+	
+	@Override
+	protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase<?, OUT, ?> translateToDataFlow(Operator<IN> input) {
+
+		String name = getName() != null ? getName() : function.getClass().getName();
+		
+		// distinguish between grouped reduce and non-grouped reduce
+		if (grouper == null) {
+			// non grouped reduce
+			UnaryOperatorInformation<IN, OUT> operatorInfo = new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType());
+			GroupReduceOperatorBase<IN, OUT, GroupReduceFunction<IN, OUT>> po =
+					new GroupReduceOperatorBase<IN, OUT, GroupReduceFunction<IN, OUT>>(function, operatorInfo, new int[0], name);
+
+			po.setCombinable(combinable);
+			// set input
+			po.setInput(input);
+			// the degree of parallelism for a non grouped reduce can only be 1
+			po.setDegreeOfParallelism(1);
+			return po;
+		}
+	
+		if (grouper.getKeys() instanceof Keys.SelectorFunctionKeys) {
+		
+			@SuppressWarnings("unchecked")
+			Keys.SelectorFunctionKeys<IN, ?> selectorKeys = (Keys.SelectorFunctionKeys<IN, ?>) grouper.getKeys();
+			
+			PlanUnwrappingReduceGroupOperator<IN, OUT, ?> po = translateSelectorFunctionReducer(
+							selectorKeys, function, getInputType(), getResultType(), name, input, isCombinable());
+			
+			po.setDegreeOfParallelism(this.getParallelism());
+			
+			return po;
+		}
+		else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) {
+
+			int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions();
+			UnaryOperatorInformation<IN, OUT> operatorInfo = new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType());
+			GroupReduceOperatorBase<IN, OUT, GroupReduceFunction<IN, OUT>> po =
+					new GroupReduceOperatorBase<IN, OUT, GroupReduceFunction<IN, OUT>>(function, operatorInfo, logicalKeyPositions, name);
+
+			po.setCombinable(combinable);
+			po.setInput(input);
+			po.setDegreeOfParallelism(this.getParallelism());
+			
+			// set group order
+			if (grouper instanceof SortedGrouping) {
+				SortedGrouping<IN> sortedGrouper = (SortedGrouping<IN>) grouper;
+								
+				int[] sortKeyPositions = sortedGrouper.getGroupSortKeyPositions();
+				Order[] sortOrders = sortedGrouper.getGroupSortOrders();
+				
+				Ordering o = new Ordering();
+				for(int i=0; i < sortKeyPositions.length; i++) {
+					o.appendOrdering(sortKeyPositions[i], null, sortOrders[i]);
+				}
+				po.setGroupOrder(o);
+			}
+			
+			return po;
+		}
+		else if (grouper.getKeys() instanceof Keys.ExpressionKeys) {
+
+			int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions();
+			UnaryOperatorInformation<IN, OUT> operatorInfo = new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType());
+			GroupReduceOperatorBase<IN, OUT, GenericGroupReduce<IN, OUT>> po =
+					new GroupReduceOperatorBase<IN, OUT, GenericGroupReduce<IN, OUT>>(function, operatorInfo, logicalKeyPositions, name);
+
+			po.setCombinable(combinable);
+			po.setInput(input);
+			po.setDegreeOfParallelism(this.getParallelism());
+			
+			return po;
+		}
+		else {
+			throw new UnsupportedOperationException("Unrecognized key type.");
+		}
+		
+	}
+	
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static <IN, OUT, K> PlanUnwrappingReduceGroupOperator<IN, OUT, K> translateSelectorFunctionReducer(
+			Keys.SelectorFunctionKeys<IN, ?> rawKeys, GroupReduceFunction<IN, OUT> function,
+			TypeInformation<IN> inputType, TypeInformation<OUT> outputType, String name, Operator<IN> input,
+			boolean combinable)
+	{
+		@SuppressWarnings("unchecked")
+		final Keys.SelectorFunctionKeys<IN, K> keys = (Keys.SelectorFunctionKeys<IN, K>) rawKeys;
+		
+		TypeInformation<Tuple2<K, IN>> typeInfoWithKey = new TupleTypeInfo<Tuple2<K, IN>>(keys.getKeyType(), inputType);
+		
+		KeyExtractingMapper<IN, K> extractor = new KeyExtractingMapper<IN, K>(keys.getKeyExtractor());
+		
+		PlanUnwrappingReduceGroupOperator<IN, OUT, K> reducer = new PlanUnwrappingReduceGroupOperator<IN, OUT, K>(function, keys, name, outputType, typeInfoWithKey, combinable);
+		
+		MapOperatorBase<IN, Tuple2<K, IN>, MapFunction<IN, Tuple2<K, IN>>> mapper = new MapOperatorBase<IN, Tuple2<K, IN>, MapFunction<IN, Tuple2<K, IN>>>(extractor, new UnaryOperatorInformation<IN, Tuple2<K, IN>>(inputType, typeInfoWithKey), "Key Extractor");
+
+		reducer.setInput(mapper);
+		mapper.setInput(input);
+		
+		// set the mapper's parallelism to the input parallelism to make sure it is chained
+		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		
+		return reducer;
+	}
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22b24f20/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
----------------------------------------------------------------------
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 37e74ef..3223f4d 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 @@ import org.apache.flink.api.java.DataSet;
  * Grouping is an intermediate step for a transformation on a grouped DataSet.<br/>
  * The following transformation can be applied on Grouping:
  * <ul>
- * 	<li>{@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.ReduceFunction)},</li>
- * <li>{@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.GroupReduceFunction)}, and</li>
+ * 	<li>{@link UnsortedGrouping#reduce(org.apache.flink.api.java.functions.RichReduceFunction)},</li>
+ * <li>{@link UnsortedGrouping#reduceGroup(org.apache.flink.api.java.functions.RichGroupReduceFunction)}, and</li>
  * <li>{@link UnsortedGrouping#aggregate(org.apache.flink.api.java.aggregation.Aggregations, int)}.</li>
  * </ul>
  *