You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2015/10/09 16:20:50 UTC

[1/6] flink git commit: [FLINK-2576] [javaAPI] [scalaAPI] Restored binary compatibility for DataSet (inner) join.

Repository: flink
Updated Branches:
  refs/heads/master 04558574c -> 1272cd5d0


[FLINK-2576] [javaAPI] [scalaAPI] Restored binary compatibility for DataSet (inner) join.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1272cd5d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1272cd5d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1272cd5d

Branch: refs/heads/master
Commit: 1272cd5d0c477fea90753dcc1b39736db56ac757
Parents: b00c1d7
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Oct 9 12:12:29 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Oct 9 16:19:21 2015 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/api/java/DataSet.java |  80 +++----
 .../flink/api/java/operators/JoinOperator.java  | 128 ++++++++++
 .../operators/join/InnerJoinOperatorSets.java   | 154 ------------
 .../java/operators/join/JoinOperatorSets.java   | 235 -------------------
 .../operators/join/JoinOperatorSetsBase.java    | 235 +++++++++++++++++++
 .../org/apache/flink/api/scala/DataSet.scala    |  16 +-
 .../apache/flink/api/scala/joinDataSet.scala    |   8 +-
 7 files changed, 415 insertions(+), 441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
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 6c8df21..08b233f 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
@@ -63,7 +63,7 @@ import org.apache.flink.api.java.operators.FlatMapOperator;
 import org.apache.flink.api.java.operators.GroupCombineOperator;
 import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.operators.join.InnerJoinOperatorSets;
+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.MapPartitionOperator;
@@ -75,7 +75,7 @@ import org.apache.flink.api.java.operators.SortPartitionOperator;
 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.join.JoinOperatorSets;
+import org.apache.flink.api.java.operators.join.JoinOperatorSetsBase;
 import org.apache.flink.api.java.operators.join.JoinType;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -729,17 +729,17 @@ public abstract class DataSet<T> {
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
 	 *   joining elements into one DataSet.<br/>
 	 * 
-	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
+	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
-	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see InnerJoinOperatorSets
+	 * @see JoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> InnerJoinOperatorSets<T, R> join(DataSet<R> other) {
-		return new InnerJoinOperatorSets<T, R>(this, other);
+	public <R> JoinOperatorSets<T, R> join(DataSet<R> other) {
+		return new JoinOperatorSets<T, R>(this, other);
 	}
 	
 	/**
@@ -748,19 +748,19 @@ public abstract class DataSet<T> {
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
 	 *   joining elements into one DataSet.<br/>
 	 * 
-	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
+	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @param strategy The strategy that should be used execute the join. If {@code null} is given, then the
 	 *                 optimizer will pick the join strategy.
-	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see InnerJoinOperatorSets
+	 * @see JoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> InnerJoinOperatorSets<T, R> join(DataSet<R> other, JoinHint strategy) {
-		return new InnerJoinOperatorSets<T, R>(this, other, strategy);
+	public <R> JoinOperatorSets<T, R> join(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSets<T, R>(this, other, strategy);
 	}
 
 	/**
@@ -770,18 +770,18 @@ public abstract class DataSet<T> {
 	 *   joining elements into one DataSet.<br/>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
 	 *   smaller than the first one.<br/>
-	 * This method returns a {@link InnerJoinOperatorSets} on which
-	 *   {@link InnerJoinOperatorSets#where(String...)} needs to be called to define the join key of the first
+	 * This method returns a {@link JoinOperatorSets} on which
+	 *   {@link JoinOperatorSets#where(String...)} needs to be called to define the join key of the first
 	 *   joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
-	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see InnerJoinOperatorSets
+	 * @see JoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> InnerJoinOperatorSets<T, R> joinWithTiny(DataSet<R> other) {
-		return new InnerJoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_SECOND);
+	public <R> JoinOperatorSets<T, R> joinWithTiny(DataSet<R> other) {
+		return new JoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_SECOND);
 	}
 	
 	/**
@@ -791,17 +791,17 @@ public abstract class DataSet<T> {
 	 *   joining elements into one DataSet.<br/>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
 	 *   larger than the first one.<br/>
-	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
+	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 * 
-	 * @see InnerJoinOperatorSets
+	 * @see JoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> InnerJoinOperatorSets<T, R> joinWithHuge(DataSet<R> other) {
-		return new InnerJoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_FIRST);
+	public <R> JoinOperatorSets<T, R> joinWithHuge(DataSet<R> other) {
+		return new JoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_FIRST);
 	}
 
 	/**
@@ -816,11 +816,11 @@ public abstract class DataSet<T> {
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> leftOuterJoin(DataSet<R> other) {
-		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.LEFT_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> leftOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSetsBase<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.LEFT_OUTER);
 	}
 
 	/**
@@ -837,11 +837,11 @@ public abstract class DataSet<T> {
 	 *                 optimizer will pick the join strategy.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> leftOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSets<>(this, other, strategy, JoinType.LEFT_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> leftOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.LEFT_OUTER);
 	}
 
 	/**
@@ -856,11 +856,11 @@ public abstract class DataSet<T> {
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> rightOuterJoin(DataSet<R> other) {
-		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.RIGHT_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> rightOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSetsBase<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.RIGHT_OUTER);
 	}
 
 	/**
@@ -877,11 +877,11 @@ public abstract class DataSet<T> {
 	 *                 optimizer will pick the join strategy.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> rightOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSets<>(this, other, strategy, JoinType.RIGHT_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> rightOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.RIGHT_OUTER);
 	}
 
 	/**
@@ -896,11 +896,11 @@ public abstract class DataSet<T> {
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> fullOuterJoin(DataSet<R> other) {
-		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.FULL_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> fullOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSetsBase<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.FULL_OUTER);
 	}
 
 	/**
@@ -917,11 +917,11 @@ public abstract class DataSet<T> {
 	 *                 optimizer will pick the join strategy.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 *
-	 * @see JoinOperatorSets
+	 * @see org.apache.flink.api.java.operators.join.JoinOperatorSetsBase
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> fullOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSets<>(this, other, strategy, JoinType.FULL_OUTER);
+	public <R> JoinOperatorSetsBase<T, R> fullOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.FULL_OUTER);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
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 7f15f9e..3679b87 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
@@ -42,10 +42,12 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
+import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.SemanticPropUtil;
 import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder;
 import org.apache.flink.api.java.operators.Keys.ExpressionKeys;
 import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException;
+import org.apache.flink.api.java.operators.join.JoinOperatorSetsBase;
 import org.apache.flink.api.java.operators.join.JoinType;
 import org.apache.flink.api.java.operators.join.JoinFunctionAssigner;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
@@ -842,6 +844,132 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 //			throw new UnsupportedOperationException("RightSemiJoin operator currently not supported.");
 //		}
 //	}
+
+	/**
+	 * Intermediate step of a Join transformation. <br/>
+	 * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
+	 * {@link JoinOperatorSets#where(int...)} or
+	 * {@link JoinOperatorSets#where(org.apache.flink.api.java.functions.KeySelector)}.
+	 *
+	 * @param <I1> The type of the first input DataSet of the Join transformation.
+	 * @param <I2> The type of the second input DataSet of the Join transformation.
+	 */
+	public static final class JoinOperatorSets<I1, I2> extends JoinOperatorSetsBase<I1, I2> {
+
+		public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
+			super(input1, input2);
+		}
+
+		public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
+			super(input1, input2, hint);
+		}
+
+		/**
+		 * {@inheritDoc}
+		 *
+		 * @return An incomplete Join transformation.
+		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
+		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
+		 *           to continue the Join.
+		 */
+		@Override
+		public JoinOperatorSetsPredicate where(int... fields) {
+			return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+		}
+
+		/**
+		 * {@inheritDoc}
+		 *
+		 * @return An incomplete Join transformation.
+		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
+		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
+		 *           to continue the Join.
+		 */
+		@Override
+		public JoinOperatorSetsPredicate where(String... fields) {
+			return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+		}
+
+		/**
+		 * {@inheritDoc}
+		 *
+		 * @return An incomplete Join transformation.
+		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
+		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
+		 *           to continue the Join.
+		 */
+		@Override
+		public <K> JoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+			return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
+		}
+
+
+		/**
+		 * Intermediate step of a Join transformation. <br/>
+		 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
+		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
+		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}.
+		 */
+		public class JoinOperatorSetsPredicate extends JoinOperatorSetsPredicateBase {
+
+			private JoinOperatorSetsPredicate(Keys<I1> keys1) {
+				super(keys1);
+			}
+
+			/**
+			 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
+			 * {@link DataSet} that should be used as join keys.<br/>
+			 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+			 * <p/>
+			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+			 * the element of the first input being the first field of the tuple and the element of the
+			 * second input being the second field of the tuple.
+			 *
+			 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
+			 * @return A DefaultJoin that represents the joined DataSet.
+			 */
+			@Override
+			public DefaultJoin<I1, I2> equalTo(int... fields) {
+				return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
+			}
+
+			/**
+			 * Continues a Join transformation and defines the fields of the second join
+			 * {@link DataSet} that should be used as join keys.<br/>
+			 * <p/>
+			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+			 * the element of the first input being the first field of the tuple and the element of the
+			 * second input being the second field of the tuple.
+			 *
+			 * @param fields The fields of the second join DataSet that should be used as keys.
+			 * @return A DefaultJoin that represents the joined DataSet.
+			 */
+			@Override
+			public DefaultJoin<I1, I2> equalTo(String... fields) {
+				return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
+			}
+
+			/**
+			 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+			 * The KeySelector function is called for each element of the second DataSet and extracts a single
+			 * key value on which the DataSet is joined. </br>
+			 * <p/>
+			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+			 * the element of the first input being the first field of the tuple and the element of the
+			 * second input being the second field of the tuple.
+			 *
+			 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
+			 * @return A DefaultJoin that represents the joined DataSet.
+			 */
+			@Override
+			public <K> DefaultJoin<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
+				TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+				return createDefaultJoin(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
+			}
+		}
+	}
+
 	
 	// --------------------------------------------------------------------------------------------
 	//  default join functions

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
deleted file mode 100644
index e0e15ca..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
+++ /dev/null
@@ -1,154 +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.operators.join;
-
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-/**
- * Intermediate step of a Join transformation. <br/>
- * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
- * {@link InnerJoinOperatorSets#where(int...)} or
- * {@link InnerJoinOperatorSets#where(KeySelector)}.
- *
- * @param <I1> The type of the first input DataSet of the Join transformation.
- * @param <I2> The type of the second input DataSet of the Join transformation.
- */
-public final class InnerJoinOperatorSets<I1, I2> extends JoinOperatorSets<I1, I2> {
-
-	public InnerJoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
-		super(input1, input2);
-	}
-
-	public InnerJoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
-		super(input1, input2, hint);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 *
-	 * @return An incomplete Join transformation.
-	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 */
-	@Override
-	public InnerJoinOperatorSetsPredicate where(int... fields) {
-		return new InnerJoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
-	}
-
-	/**
-	 * {@inheritDoc}
-	 *
-	 * @return An incomplete Join transformation.
-	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 */
-	@Override
-	public InnerJoinOperatorSetsPredicate where(String... fields) {
-		return new InnerJoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
-	}
-
-	/**
-	 * {@inheritDoc}
-	 *
-	 * @return An incomplete Join transformation.
-	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 */
-	@Override
-	public <K> InnerJoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
-		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
-		return new InnerJoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
-	}
-
-
-	/**
-	 * Intermediate step of a Join transformation. <br/>
-	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
-	 * {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
-	 * {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}.
-	 */
-	public class InnerJoinOperatorSetsPredicate extends JoinOperatorSetsPredicate {
-
-		private InnerJoinOperatorSetsPredicate(Keys<I1> keys1) {
-			super(keys1);
-		}
-
-		/**
-		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
-		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-		 * <p/>
-		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
-		 * the element of the first input being the first field of the tuple and the element of the
-		 * second input being the second field of the tuple.
-		 *
-		 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
-		 * @return A DefaultJoin that represents the joined DataSet.
-		 */
-		@Override
-		public DefaultJoin<I1, I2> equalTo(int... fields) {
-			return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
-		}
-
-		/**
-		 * Continues a Join transformation and defines the fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
-		 * <p/>
-		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
-		 * the element of the first input being the first field of the tuple and the element of the
-		 * second input being the second field of the tuple.
-		 *
-		 * @param fields The fields of the second join DataSet that should be used as keys.
-		 * @return A DefaultJoin that represents the joined DataSet.
-		 */
-		@Override
-		public DefaultJoin<I1, I2> equalTo(String... fields) {
-			return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
-		}
-
-		/**
-		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
-		 * The KeySelector function is called for each element of the second DataSet and extracts a single
-		 * key value on which the DataSet is joined. </br>
-		 * <p/>
-		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
-		 * the element of the first input being the first field of the tuple and the element of the
-		 * second input being the second field of the tuple.
-		 *
-		 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
-		 * @return A DefaultJoin that represents the joined DataSet.
-		 */
-		@Override
-		public <K> DefaultJoin<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
-			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
-			return createDefaultJoin(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
deleted file mode 100644
index 705952c..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
+++ /dev/null
@@ -1,235 +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.operators.join;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
-import org.apache.flink.api.java.operators.JoinOperator.EquiJoin;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-/**
- * Intermediate step of an Outer Join transformation. <br/>
- * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
- * {@link JoinOperatorSets#where(int...)} or
- * {@link JoinOperatorSets#where(KeySelector)}.
- *
- * @param <I1> The type of the first input DataSet of the Join transformation.
- * @param <I2> The type of the second input DataSet of the Join transformation.
- */
-public class JoinOperatorSets<I1, I2> {
-
-	protected final DataSet<I1> input1;
-	protected final DataSet<I2> input2;
-
-	protected final JoinHint joinHint;
-	protected final JoinType joinType;
-
-	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
-		this(input1, input2, JoinHint.OPTIMIZER_CHOOSES);
-	}
-
-	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
-		this(input1, input2, hint, JoinType.INNER);
-	}
-
-	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint, JoinType type) {
-		if (input1 == null || input2 == null) {
-			throw new NullPointerException();
-		}
-
-		this.input1 = input1;
-		this.input2 = input2;
-		this.joinHint = hint;
-		this.joinType = type;
-	}
-
-	/**
-	 * Continues a Join transformation. <br/>
-	 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br/>
-	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-	 *
-	 * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys.
-	 * @return An incomplete Join transformation.
-	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 *
-	 * @see Tuple
-	 * @see DataSet
-	 */
-	public JoinOperatorSetsPredicate where(int... fields) {
-		return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
-	}
-
-	/**
-	 * Continues a Join transformation. <br/>
-	 * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields
-	 * are the names of member fields of the underlying type of the data set.
-	 *
-	 * @param fields The  fields of the first join DataSets that should be used as keys.
-	 * @return An incomplete Join transformation.
-	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 *
-	 * @see Tuple
-	 * @see DataSet
-	 */
-	public JoinOperatorSetsPredicate where(String... fields) {
-		return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
-	}
-
-	/**
-	 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.</br>
-	 * The KeySelector function is called for each element of the first DataSet and extracts a single
-	 * key value on which the DataSet is joined. </br>
-	 *
-	 * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined.
-	 * @return An incomplete Join transformation.
-	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
-	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
-	 *           to continue the Join.
-	 *
-	 * @see KeySelector
-	 * @see DataSet
-	 */
-	public <K> JoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
-		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
-		return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
-	}
-
-
-	/**
-	 * Intermediate step of a Join transformation. <br/>
-	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
-	 * {@link JoinOperatorSetsPredicate#equalTo(int...)} or
-	 * {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}.
-	 *
-	 */
-	public class JoinOperatorSetsPredicate {
-
-		protected final Keys<I1> keys1;
-
-		protected JoinOperatorSetsPredicate(Keys<I1> keys1) {
-			if (keys1 == null) {
-				throw new NullPointerException();
-			}
-
-			if (keys1.isEmpty()) {
-				throw new InvalidProgramException("The join keys must not be empty.");
-			}
-
-			this.keys1 = keys1;
-		}
-
-		/**
-		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
-		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-		 *
-		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
-		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
-		 *
-		 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
-		 * @return A JoinFunctionAssigner.
-		 */
-		public JoinFunctionAssigner<I1, I2> equalTo(int... fields) {
-			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
-		}
-
-		/**
-		 * Continues a Join transformation and defines the fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
-		 *
-		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
-		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
-		 *
-		 * @param fields The fields of the second join DataSet that should be used as keys.
-		 * @return A JoinFunctionAssigner.
-		 */
-		public JoinFunctionAssigner<I1, I2> equalTo(String... fields) {
-			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
-		}
-
-		/**
-		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
-		 * The KeySelector function is called for each element of the second DataSet and extracts a single
-		 * key value on which the DataSet is joined. </br>
-		 *
-		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
-		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
-		 *
-		 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
-		 * @return A JoinFunctionAssigner.
-		 */
-		public <K> JoinFunctionAssigner<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
-			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
-			return createJoinFunctionAssigner(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
-		}
-
-		protected JoinFunctionAssigner<I1, I2> createJoinFunctionAssigner(Keys<I2> keys2) {
-			DefaultJoin<I1, I2> join = createDefaultJoin(keys2);
-			return new DefaultJoinFunctionAssigner(join);
-		}
-
-		protected DefaultJoin<I1, I2> createDefaultJoin(Keys<I2> keys2) {
-			if (keys2 == null) {
-				throw new NullPointerException("The join keys may not be null.");
-			}
-
-			if (keys2.isEmpty()) {
-				throw new InvalidProgramException("The join keys may not be empty.");
-			}
-
-			try {
-				keys1.areCompatible(keys2);
-			} catch (Keys.IncompatibleKeysException e) {
-				throw new InvalidProgramException("The pair of join keys are not compatible with each other.",e);
-			}
-			return new DefaultJoin<>(input1, input2, keys1, keys2, joinHint, Utils.getCallLocationName(4), joinType);
-		}
-
-		private class DefaultJoinFunctionAssigner implements JoinFunctionAssigner<I1, I2> {
-
-			private final DefaultJoin<I1, I2> defaultJoin;
-
-			public DefaultJoinFunctionAssigner(DefaultJoin<I1, I2> defaultJoin) {
-				this.defaultJoin = defaultJoin;
-			}
-
-			public <R> EquiJoin<I1, I2, R> with(JoinFunction<I1, I2, R> joinFunction) {
-				return defaultJoin.with(joinFunction);
-			}
-
-			public <R> EquiJoin<I1, I2, R> with(FlatJoinFunction<I1, I2, R> joinFunction) {
-				return defaultJoin.with(joinFunction);
-			}
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
new file mode 100644
index 0000000..6f00cdb
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
@@ -0,0 +1,235 @@
+/*
+ * 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.join;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
+import org.apache.flink.api.java.operators.JoinOperator.EquiJoin;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+/**
+ * Intermediate step of an Outer Join transformation. <br/>
+ * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
+ * {@link JoinOperatorSetsBase#where(int...)} or
+ * {@link JoinOperatorSetsBase#where(KeySelector)}.
+ *
+ * @param <I1> The type of the first input DataSet of the Join transformation.
+ * @param <I2> The type of the second input DataSet of the Join transformation.
+ */
+public class JoinOperatorSetsBase<I1, I2> {
+
+	protected final DataSet<I1> input1;
+	protected final DataSet<I2> input2;
+
+	protected final JoinHint joinHint;
+	protected final JoinType joinType;
+
+	public JoinOperatorSetsBase(DataSet<I1> input1, DataSet<I2> input2) {
+		this(input1, input2, JoinHint.OPTIMIZER_CHOOSES);
+	}
+
+	public JoinOperatorSetsBase(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
+		this(input1, input2, hint, JoinType.INNER);
+	}
+
+	public JoinOperatorSetsBase(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint, JoinType type) {
+		if (input1 == null || input2 == null) {
+			throw new NullPointerException();
+		}
+
+		this.input1 = input1;
+		this.input2 = input2;
+		this.joinHint = hint;
+		this.joinType = type;
+	}
+
+	/**
+	 * Continues a Join transformation. <br/>
+	 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br/>
+	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+	 *
+	 * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or
+	 *           {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see Tuple
+	 * @see DataSet
+	 */
+	public JoinOperatorSetsPredicateBase where(int... fields) {
+		return new JoinOperatorSetsPredicateBase(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * Continues a Join transformation. <br/>
+	 * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields
+	 * are the names of member fields of the underlying type of the data set.
+	 *
+	 * @param fields The  fields of the first join DataSets that should be used as keys.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or
+	 *           {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see Tuple
+	 * @see DataSet
+	 */
+	public JoinOperatorSetsPredicateBase where(String... fields) {
+		return new JoinOperatorSetsPredicateBase(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.</br>
+	 * The KeySelector function is called for each element of the first DataSet and extracts a single
+	 * key value on which the DataSet is joined. </br>
+	 *
+	 * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or
+	 *           {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see KeySelector
+	 * @see DataSet
+	 */
+	public <K> JoinOperatorSetsPredicateBase where(KeySelector<I1, K> keySelector) {
+		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+		return new JoinOperatorSetsPredicateBase(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
+	}
+
+
+	/**
+	 * Intermediate step of a Join transformation. <br/>
+	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
+	 * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or
+	 * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}.
+	 *
+	 */
+	public class JoinOperatorSetsPredicateBase {
+
+		protected final Keys<I1> keys1;
+
+		protected JoinOperatorSetsPredicateBase(Keys<I1> keys1) {
+			if (keys1 == null) {
+				throw new NullPointerException();
+			}
+
+			if (keys1.isEmpty()) {
+				throw new InvalidProgramException("The join keys must not be empty.");
+			}
+
+			this.keys1 = keys1;
+		}
+
+		/**
+		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public JoinFunctionAssigner<I1, I2> equalTo(int... fields) {
+			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines the fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param fields The fields of the second join DataSet that should be used as keys.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public JoinFunctionAssigner<I1, I2> equalTo(String... fields) {
+			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+		 * The KeySelector function is called for each element of the second DataSet and extracts a single
+		 * key value on which the DataSet is joined. </br>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public <K> JoinFunctionAssigner<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+			return createJoinFunctionAssigner(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
+		}
+
+		protected JoinFunctionAssigner<I1, I2> createJoinFunctionAssigner(Keys<I2> keys2) {
+			DefaultJoin<I1, I2> join = createDefaultJoin(keys2);
+			return new DefaultJoinFunctionAssigner(join);
+		}
+
+		protected DefaultJoin<I1, I2> createDefaultJoin(Keys<I2> keys2) {
+			if (keys2 == null) {
+				throw new NullPointerException("The join keys may not be null.");
+			}
+
+			if (keys2.isEmpty()) {
+				throw new InvalidProgramException("The join keys may not be empty.");
+			}
+
+			try {
+				keys1.areCompatible(keys2);
+			} catch (Keys.IncompatibleKeysException e) {
+				throw new InvalidProgramException("The pair of join keys are not compatible with each other.",e);
+			}
+			return new DefaultJoin<>(input1, input2, keys1, keys2, joinHint, Utils.getCallLocationName(4), joinType);
+		}
+
+		private class DefaultJoinFunctionAssigner implements JoinFunctionAssigner<I1, I2> {
+
+			private final DefaultJoin<I1, I2> defaultJoin;
+
+			public DefaultJoinFunctionAssigner(DefaultJoin<I1, I2> defaultJoin) {
+				this.defaultJoin = defaultJoin;
+			}
+
+			public <R> EquiJoin<I1, I2, R> with(JoinFunction<I1, I2, R> joinFunction) {
+				return defaultJoin.with(joinFunction);
+			}
+
+			public <R> EquiJoin<I1, I2, R> with(FlatJoinFunction<I1, I2, R> joinFunction) {
+				return defaultJoin.with(joinFunction);
+			}
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 306a15b..0f23bee 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -876,29 +876,29 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    *     }
    * }}}
    */
-  def join[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
-    new UnfinishedInnerJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES)
+  def join[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES)
 
   /**
    * Special [[join]] operation for explicitly telling the system what join strategy to use. If
    * null is given as the join strategy, then the optimizer will pick the strategy.
    */
-  def join[O](other: DataSet[O], strategy: JoinHint): UnfinishedInnerJoinOperation[T, O] =
-    new UnfinishedInnerJoinOperation(this, other, strategy)
+  def join[O](other: DataSet[O], strategy: JoinHint): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperation(this, other, strategy)
   
   /**
    * Special [[join]] operation for explicitly telling the system that the right side is assumed
    * to be a lot smaller than the left side of the join.
    */
-  def joinWithTiny[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
-    new UnfinishedInnerJoinOperation(this, other, JoinHint.BROADCAST_HASH_SECOND)
+  def joinWithTiny[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperation(this, other, JoinHint.BROADCAST_HASH_SECOND)
 
   /**
    * Special [[join]] operation for explicitly telling the system that the left side is assumed
    * to be a lot smaller than the right side of the join.
    */
-  def joinWithHuge[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
-    new UnfinishedInnerJoinOperation(this, other, JoinHint.BROADCAST_HASH_FIRST)
+  def joinWithHuge[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperation(this, other, JoinHint.BROADCAST_HASH_FIRST)
 
   /**
    * Creates a new DataSet by performing a full outer join of `this` DataSet

http://git-wip-us.apache.org/repos/asf/flink/blob/1272cd5d/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index f57fc25..d94b809 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -212,7 +212,7 @@ class JoinDataSet[L, R](
   }
 }
 
-private[flink] abstract class UnfinishedJoinOperation[L, R, O <: JoinFunctionAssigner[L, R]](
+private[flink] abstract class UnfinishedJoinOperationBase[L, R, O <: JoinFunctionAssigner[L, R]](
     leftSet: DataSet[L],
     rightSet: DataSet[R],
     val joinHint: JoinHint,
@@ -261,11 +261,11 @@ private[flink] abstract class UnfinishedJoinOperation[L, R, O <: JoinFunctionAss
  * @tparam L The type of the left input of the join.
  * @tparam R The type of the right input of the join.
  */
-class UnfinishedInnerJoinOperation[L, R](
+class UnfinishedJoinOperation[L, R](
     leftSet: DataSet[L],
     rightSet: DataSet[R],
     joinHint: JoinHint)
-  extends UnfinishedJoinOperation[L, R, JoinDataSet[L, R]](
+  extends UnfinishedJoinOperationBase[L, R, JoinDataSet[L, R]](
     leftSet, rightSet, joinHint, JoinType.INNER) {
 
   override def createJoinFunctionAssigner(leftKey: Keys[L], rightKey: Keys[R]) = {
@@ -297,7 +297,7 @@ class UnfinishedOuterJoinOperation[L, R](
     rightSet: DataSet[R],
     joinHint: JoinHint,
     joinType: JoinType)
-  extends UnfinishedJoinOperation[L, R, JoinFunctionAssigner[L, R]](
+  extends UnfinishedJoinOperationBase[L, R, JoinFunctionAssigner[L, R]](
     leftSet, rightSet, joinHint, joinType) {
 
   override def createJoinFunctionAssigner(leftKey: Keys[L], rightKey: Keys[R]):


[2/6] flink git commit: [FLINK-2576] [javaAPI] [scalaAPI] [optimizer] Add outerJoin to DataSet API (Java, Scala) and optimizer.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
new file mode 100644
index 0000000..ebd1ddf
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
@@ -0,0 +1,605 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+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.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class OuterJoinITCase extends MultipleProgramsTestBase {
+
+	public OuterJoinITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testUDFLeftOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.leftOuterJoin(ds2)
+						.where(0)
+						.equalTo(0)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello,Hallo Welt wie\n" +
+				"Hello world,null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testUDFRightOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.rightOuterJoin(ds2)
+						.where(1)
+						.equalTo(1)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"null,Hallo Welt wie\n" +
+				"Hello world,Hallo Welt\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testUDFFullOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(0)
+						.equalTo(2)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "null,Hallo\n" +
+				"Hi,Hallo Welt\n" +
+				"Hello,Hallo Welt wie\n" +
+				"Hello world,null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with multiple key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(0, 1)
+						.equalTo(0, 4)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,null\n" +
+				"null,Hallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithBroadcastSet() throws Exception {
+		/*
+		 * Join with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple3<String, String, Integer>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(1)
+						.equalTo(4)
+						.with(new T3T5BCJoin())
+						.withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<String, String, Integer>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo,55\n" +
+				"Hi,Hallo Welt wie,55\n" +
+				"Hello,Hallo Welt,55\n" +
+				"Hello world,Hallo Welt,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector() throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(new KeySelector1())
+						.equalTo(0)
+						.with(new CustT3Join());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hi\n" +
+				"Hello,Hello\n" +
+				"Hello world,Hello\n" +
+				"null,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	public static class KeySelector1 implements KeySelector<CustomType, Integer> {
+		@Override
+		public Integer getKey(CustomType value) {
+			return value.myInt;
+		}
+	}
+
+
+	@Test
+	public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
+			throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(1)
+						.equalTo(new KeySelector2())
+						.with(new T3CustJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "null,Hi\n" +
+				"Hi,Hello\n" +
+				"Hello,Hello world\n" +
+				"Hello world,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	public static class KeySelector2 implements KeySelector<CustomType, Long> {
+		@Override
+		public Long getKey(CustomType value) {
+			return value.myLong;
+		}
+	}
+
+	@Test
+	public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception {
+		/*
+		 * UDF Join on tuples with tuple-returning key selectors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(new KeySelector3()) //0, 1
+						.equalTo(new KeySelector4()) // 0, 4
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,null\n" +
+				"null,Hallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	public static class KeySelector3 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+			return new Tuple2<>(t.f0, t.f1);
+		}
+	}
+
+	public static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected using a string)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber")
+						.equalTo("f6")
+						.with(new ProjectBothFunction());
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected as an integer)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber")
+						.equalTo(6) // <--- difference!
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exception {
+		/*
+		 * selecting multiple fields using expression language
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber", "number", "str")
+						.equalTo("f6", "f0", "f1")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTuple() throws Exception {
+		/*
+		 * nested into tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber", "number", "nestedTupleWithCustom.f0")
+						.equalTo("f6", "f0", "f2")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTupleIntoPojo() throws Exception {
+		/*
+		 * nested into tuple into pojo
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedTupleWithCustom.f0", "nestedTupleWithCustom.f1.myInt", "nestedTupleWithCustom.f1.myLong")
+						.equalTo("f2", "f3", "f4")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNonPojoToVerifyFullTupleKeys() throws Exception {
+		/*
+		 * Non-POJO test to verify that full-tuple keys are working.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(0)
+						.equalTo("f0.f0", "f0.f1") // key is now Tuple2<Integer, Integer>
+						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	@Test
+	public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception {
+		/*
+		 * Non-POJO test to verify "nested" tuple-element selection.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("f0.f0")
+						.equalTo("f0.f0") // key is now Integer from Tuple2<Integer, Integer>
+						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testFullPojoWithFullTuple() throws Exception {
+		/*
+		 * full pojo with full tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env);
+		DataSet<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("*")
+						.equalTo("*")
+						.with(new ProjectBothFunction<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
+				"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +
+				"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Integer> ds2 = env.fromElements(1, 2);
+
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Integer>> joinDs = ds1
+				.fullOuterJoin(ds2)
+				.where(0)
+				.equalTo("*")
+				.with(new ProjectBothFunction<Tuple3<Integer, Long, String>, Integer>())
+				.returns("Tuple2<java.lang.Object,java.lang.Object>");
+
+		List<Tuple2<Tuple3<Integer, Long, String>, Integer>> result = joinDs.collect();
+
+		String expected = "(1,1,Hi),1\n" +
+				"(2,2,Hello),2\n" +
+				"(3,2,Hello world),null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType2() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ds1 = env.fromElements(1, 2);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+
+		DataSet<Tuple2<Integer, Tuple3<Integer, Long, String>>> joinDs = ds1
+				.fullOuterJoin(ds2)
+				.where("*")
+				.equalTo(0)
+				.with(new ProjectBothFunction<Integer, Tuple3<Integer, Long, String>>())
+				.returns("Tuple2<java.lang.Object,java.lang.Object>");
+
+
+		List<Tuple2<Integer, Tuple3<Integer, Long, String>>> result = joinDs.collect();
+
+		String expected = "1,(1,1,Hi)\n" +
+				"2,(2,2,Hello)\n" +
+				"null,(3,2,Hello world)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	public static class T3T5FlatJoin implements FlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<String, String>> {
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second,
+				Collector<Tuple2<String, String>> out) {
+
+			out.collect(new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.f3));
+		}
+
+	}
+
+	public static class T3T5BCJoin extends RichFlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<String, String, Integer>> {
+
+		private int broadcast;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			broadcast = sum;
+		}
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first, Tuple5<Integer, Long, Integer, String, Long> second,
+				Collector<Tuple3<String, String, Integer>> out) throws Exception {
+			out.collect(new Tuple3<>(first == null ? null : first.f2, second == null ? null : second.f3, broadcast));
+		}
+	}
+
+	public static class T3CustJoin implements JoinFunction<Tuple3<Integer, Long, String>, CustomType, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(Tuple3<Integer, Long, String> first,
+				CustomType second) {
+
+			return new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.myString);
+		}
+	}
+
+	public static class CustT3Join implements JoinFunction<CustomType, Tuple3<Integer, Long, String>, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(CustomType first, Tuple3<Integer, Long, String> second) {
+
+			return new Tuple2<>(first == null ? null : first.myString, second == null ? null : second.f2);
+		}
+	}
+
+	/**
+	 * Deliberately untyped join function, which emits a Tuple2 of the left and right side.
+	 */
+	public static class ProjectBothFunction<IN1, IN2> implements JoinFunction<IN1, IN2, Tuple2<IN1, IN2>> {
+		@Override
+		public Tuple2<IN1, IN2> join(IN1 first, IN2 second) throws Exception {
+			return new Tuple2<>(first, second);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
index cc2c81e..16c826f 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
@@ -22,7 +22,7 @@ import org.junit.Assert._
 import org.apache.flink.api.common.functions.RichJoinFunction
 import org.apache.flink.api.common.functions.RichMapFunction
 import org.apache.flink.api.common.operators.{GenericDataSinkBase, SingleInputSemanticProperties}
-import org.apache.flink.api.common.operators.base.{JoinOperatorBase, MapOperatorBase}
+import org.apache.flink.api.common.operators.base.{InnerJoinOperatorBase, MapOperatorBase}
 import org.apache.flink.api.common.operators.util.FieldSet
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst
@@ -170,8 +170,8 @@ class SemanticPropertiesTranslationTest {
       val plan = env.createProgramPlan()
       val sink: GenericDataSinkBase[_] = plan.getDataSinks.iterator.next
 
-      val join: JoinOperatorBase[_, _, _, _] =
-        sink.getInput.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+      val join: InnerJoinOperatorBase[_, _, _, _] =
+        sink.getInput.asInstanceOf[InnerJoinOperatorBase[_, _, _, _]]
 
       val semantics = join.getSemanticProperties
       val fw11: FieldSet = semantics.getForwardingTargetFields(0, 0)
@@ -215,8 +215,8 @@ class SemanticPropertiesTranslationTest {
       val plan = env.createProgramPlan()
       val sink: GenericDataSinkBase[_] = plan.getDataSinks.iterator.next
 
-      val join: JoinOperatorBase[_, _, _, _] =
-        sink.getInput.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+      val join: InnerJoinOperatorBase[_, _, _, _] =
+        sink.getInput.asInstanceOf[InnerJoinOperatorBase[_, _, _, _]]
 
       val semantics = join.getSemanticProperties
       val fw11: FieldSet = semantics.getForwardingTargetFields(0, 0)

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala
new file mode 100644
index 0000000..c80472a
--- /dev/null
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.operators
+
+import org.apache.flink.api.common.functions.RichJoinFunction
+import org.apache.flink.api.common.io.OutputFormat
+import org.apache.flink.api.scala.operators.ScalaCsvOutputFormat.{DEFAULT_FIELD_DELIMITER, DEFAULT_LINE_DELIMITER}
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.api.scala.util.CollectionDataSets.CustomType
+import org.apache.flink.api.scala.{ExecutionEnvironment, _}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.core.fs.FileSystem.WriteMode
+import org.apache.flink.core.fs.Path
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+
+import scala.collection.JavaConverters._
+
+
+@RunWith(classOf[Parameterized])
+class OuterJoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
+  private var resultPath: String = null
+  private var expected: String = null
+  private val _tempFolder = new TemporaryFolder()
+
+  @Rule
+  def tempFolder = _tempFolder
+
+  @Before
+  def before(): Unit = {
+    resultPath = tempFolder.newFile().toURI.toString
+  }
+
+  @After
+  def after(): Unit = {
+    if (expected != null) {
+      TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath)
+    }
+  }
+
+  def writeAsCsv[T](ds: DataSet[T]) = {
+    val of = new ScalaCsvOutputFormat[Product](new Path(resultPath),
+      DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER)
+    of.setAllowNullValues(true)
+    of.setWriteMode(WriteMode.OVERWRITE)
+    ds.output(of.asInstanceOf[OutputFormat[T]])
+  }
+
+  def mapToString: ((String, String)) => (String, String) = {
+    (tuple: (String, String)) => (String.valueOf(tuple._1), String.valueOf(tuple._2))
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUDFLeftOuterJoinOnTuplesWithKeyFieldPositions {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.leftOuterJoin(ds2).where(0).equalTo(0).apply(T3T5FlatJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello,Hallo Welt wie\n" + "Hello world,null\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUDFRightOuterJoinOnTuplesWithKeyFieldPositions {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.rightOuterJoin(ds2).where(1).equalTo(1).apply(T3T5FlatJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "Hi,Hallo\n" +
+      "Hello,Hallo Welt\n" +
+      "null,Hallo Welt wie\n" +
+      "Hello world,Hallo Welt\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUDFFullOuterJoinOnTuplesWithKeyFieldPositions {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2).where(0).equalTo(2).apply(T3T5FlatJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "null,Hallo\n" + "Hi,Hallo Welt\n" + "Hello,Hallo Welt wie\n" + "Hello world,null\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUDFJoinOnTuplesWithMultipleKeyFieldPositions {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2).where(0, 1).equalTo(0, 4).apply(T3T5FlatJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,null\n" + "null,Hallo Welt wie\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testJoinWithBroadcastSet {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val intDs = CollectionDataSets.getIntDataSet(env)
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2).where(1).equalTo(4).apply(
+      new RichJoinFunction[
+        (Int, Long, String),
+        (Int, Long, Int, String, Long),
+        (String, String, Int)] {
+        private var broadcast = 41
+
+        override def open(config: Configuration) {
+          val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala
+          broadcast = ints.sum
+        }
+
+        override def join(l: (Int, Long, String),
+                          r: (Int, Long, Int, String, Long)): (String, String, Int) = {
+          (if (l == null) "null" else l._3, if (r == null) "null" else r._4, broadcast)
+        }
+      }
+    ).withBroadcastSet(intDs, "ints")
+    writeAsCsv(joinDs)
+    env.execute()
+    expected = "Hi,Hallo,55\n" +
+      "Hi,Hallo Welt wie,55\n" +
+      "Hello,Hallo Welt,55\n" +
+      "Hello world,Hallo Welt,55\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2).where(t => t.myInt).equalTo(0).apply(CustT3Join)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "Hi,Hi\n" + "Hello,Hello\n" + "Hello world,Hello\n" + "null,Hello world\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2).where(1).equalTo(t => t.myLong).apply(T3CustJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "null,Hi\n" + "Hi,Hello\n" + "Hello,Hello world\n" + "Hello world,Hello world\n"
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUDFJoinOnTuplesWithTupleReturningKeySelectors {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall5TupleDataSet(env)
+    val joinDs = ds1.fullOuterJoin(ds2)
+      .where(t => (t._1, t._2)).equalTo(t => (t._1, t._5))
+      .apply(T3T5FlatJoin)
+    writeAsCsv(joinDs.map(mapToString))
+    env.execute()
+    expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,null\n" + "null,Hallo Welt wie\n"
+  }
+
+
+  def T3T5FlatJoin: ((Int, Long, String), (Int, Long, Int, String, Long)) => (String, String) = {
+    (first, second) => {
+      (if (first == null) null else first._3, if (second == null) null else second._4)
+    }
+  }
+
+  def CustT3Join: (CustomType, (Int, Long, String)) => (String, String) = {
+    (first, second) => {
+      (if (first == null) null else first.myString, if (second == null) null else second._3)
+    }
+  }
+
+  def T3CustJoin: ((Int, Long, String), CustomType) => (String, String) = {
+    (first, second) => {
+      (if (first == null) null else first._3, if (second == null) null else second.myString)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
index 9a400c5..3121d68 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
@@ -28,7 +28,7 @@ import org.junit.Assert.fail
 import org.apache.flink.api.common.{InvalidProgramException, Plan}
 import org.apache.flink.api.common.aggregators.LongSumAggregator
 import org.apache.flink.api.common.operators.base.DeltaIterationBase
-import org.apache.flink.api.common.operators.base.JoinOperatorBase
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase
 import org.apache.flink.api.common.operators.base.MapOperatorBase
 import org.junit.Test
 
@@ -91,10 +91,10 @@ class DeltaIterationTranslationTest {
 
       val nextWorksetMapper: MapOperatorBase[_, _, _] =
         iteration.getNextWorkset.asInstanceOf[MapOperatorBase[_, _, _]]
-      val solutionSetJoin: JoinOperatorBase[_, _, _, _] =
-        iteration.getSolutionSetDelta.asInstanceOf[JoinOperatorBase[_, _, _, _]]
-      val worksetSelfJoin: JoinOperatorBase[_, _, _, _] =
-        solutionSetJoin.getFirstInput.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+      val solutionSetJoin: InnerJoinOperatorBase[_, _, _, _] =
+        iteration.getSolutionSetDelta.asInstanceOf[InnerJoinOperatorBase[_, _, _, _]]
+      val worksetSelfJoin: InnerJoinOperatorBase[_, _, _, _] =
+        solutionSetJoin.getFirstInput.asInstanceOf[InnerJoinOperatorBase[_, _, _, _]]
       val worksetMapper: MapOperatorBase[_, _, _] =
         worksetSelfJoin.getFirstInput.asInstanceOf[MapOperatorBase[_, _, _]]
 


[4/6] flink git commit: [FLINK-2576] [javaAPI] [scalaAPI] [optimizer] Add outerJoin to DataSet API (Java, Scala) and optimizer.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
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 e76e3c9..7f15f9e 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
@@ -18,7 +18,6 @@
 
 package org.apache.flink.api.java.operators;
 
-import java.security.InvalidParameterException;
 import java.util.Arrays;
 
 import com.google.common.base.Preconditions;
@@ -34,22 +33,25 @@ import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
-import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.SemanticPropUtil;
 import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder;
 import org.apache.flink.api.java.operators.Keys.ExpressionKeys;
 import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException;
+import org.apache.flink.api.java.operators.join.JoinType;
+import org.apache.flink.api.java.operators.join.JoinFunctionAssigner;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
-import org.apache.flink.api.java.operators.translation.PlanBothUnwrappingJoinOperator;
-import org.apache.flink.api.java.operators.translation.PlanLeftUnwrappingJoinOperator;
-import org.apache.flink.api.java.operators.translation.PlanRightUnwrappingJoinOperator;
+import org.apache.flink.api.java.operators.translation.TupleRightUnwrappingJoiner;
+import org.apache.flink.api.java.operators.translation.TupleLeftUnwrappingJoiner;
+import org.apache.flink.api.java.operators.translation.TupleUnwrappingJoiner;
 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;
@@ -69,18 +71,19 @@ import org.apache.flink.api.java.tuple.*;
  * @see DataSet
  */
 public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT, JoinOperator<I1, I2, OUT>> {
-	
+
 	protected final Keys<I1> keys1;
 	protected final Keys<I2> keys2;
 	
 	private final JoinHint joinHint;
-	
+	protected final JoinType joinType;
+
 	private Partitioner<?> customPartitioner;
 	
 	
-	protected JoinOperator(DataSet<I1> input1, DataSet<I2> input2, 
+	protected JoinOperator(DataSet<I1> input1, DataSet<I2> input2,
 			Keys<I1> keys1, Keys<I2> keys2,
-			TypeInformation<OUT> returnType, JoinHint hint)
+			TypeInformation<OUT> returnType, JoinHint hint, JoinType type)
 	{
 		super(input1, input2, returnType);
 		
@@ -116,7 +119,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 		this.keys1 = keys1;
 		this.keys2 = keys2;
-		this.joinHint = hint == null ? JoinHint.OPTIMIZER_CHOOSES : hint;
+		this.joinHint = hint == null ? InnerJoinOperatorBase.JoinHint.OPTIMIZER_CHOOSES : hint;
+		this.joinType = type;
 	}
 	
 	protected Keys<I1> getKeys1() {
@@ -135,6 +139,15 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	public JoinHint getJoinHint() {
 		return this.joinHint;
 	}
+
+	/**
+	 * Gets the JoinType that describes this join operation (e.g. inner, outer)
+	 *
+	 * @return The JoinType
+	 */
+	public JoinType getJoinType() {
+		return this.joinType;
+	}
 	
 	/**
 	 * Sets a custom partitioner for this join. The partitioner will be called on the join keys to determine
@@ -189,12 +202,23 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		private boolean preserve2;
 		
 		private final String joinLocationName;
-		
+
 		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
 				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName)
-		{
-			super(input1, input2, keys1, keys2, returnType, hint);
+				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName) {
+			this(input1, input2, keys1, keys2, function, returnType, hint, joinLocationName, JoinType.INNER);
+		}
+
+		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
+				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> generatedFunction, JoinFunction<I1, I2, OUT> function,
+				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName) {
+			this(input1, input2, keys1, keys2, generatedFunction, function, returnType, hint, joinLocationName, JoinType.INNER);
+		}
+
+		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
+				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> function,
+				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName, JoinType type) {
+			super(input1, input2, keys1, keys2, returnType, hint, type);
 			
 			if (function == null) {
 				throw new NullPointerException();
@@ -208,9 +232,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
 				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> generatedFunction, JoinFunction<I1, I2, OUT> function,
-				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName)
-		{
-			super(input1, input2, keys1, keys2, returnType, hint);
+				TypeInformation<OUT> returnType, JoinHint hint, String joinLocationName, JoinType type) {
+			super(input1, input2, keys1, keys2, returnType, hint, type);
 			
 			this.joinLocationName = joinLocationName;
 
@@ -282,232 +305,220 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 		@Override
 		protected JoinOperatorBase<?, ?, OUT, ?> translateToDataFlow(Operator<I1> input1, Operator<I2> input2) {
+			String name = getName() != null ? getName() : "Join at " + joinLocationName;
+
+			JoinOperatorBaseBuilder<OUT> builder = new JoinOperatorBaseBuilder<OUT>(name, joinType)
+					.withParallelism(getParallelism())
+					.withPartitioner(getPartitioner())
+					.withJoinHint(getJoinHint())
+					.withResultType(getResultType());
+
+			final boolean requiresTupleUnwrapping = keys1 instanceof Keys.SelectorFunctionKeys || keys2 instanceof Keys.SelectorFunctionKeys;
+			if (requiresTupleUnwrapping) {
+				if (keys1 instanceof Keys.SelectorFunctionKeys && keys2 instanceof Keys.SelectorFunctionKeys) {
+					// Both join sides have a key selector function, so we need to do the
+					// tuple wrapping/unwrapping on both sides.
+
+					@SuppressWarnings("unchecked")
+					Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
+					@SuppressWarnings("unchecked")
+					Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
+
+					builder = builder
+							.withUdf(new TupleUnwrappingJoiner<>(function))
+							.withWrappedInput1(input1, selectorKeys1, getInput1Type())
+							.withWrappedInput2(input2, selectorKeys2, getInput2Type());
+				} else if (keys2 instanceof Keys.SelectorFunctionKeys) {
+					// The right side of the join needs the tuple wrapping/unwrapping
+
+					@SuppressWarnings("unchecked")
+					Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
+
+					builder = builder
+							.withUdf(new TupleRightUnwrappingJoiner<>(function))
+							.withInput1(input1, getInput1Type(), keys1)
+							.withWrappedInput2(input2, selectorKeys2, getInput2Type());
+				} else {
+					// The left side of the join needs the tuple wrapping/unwrapping
 
-			String name = getName() != null ? getName() : "Join at "+joinLocationName;
+					@SuppressWarnings("unchecked")
+					Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
 
-			final JoinOperatorBase<?, ?, OUT, ?> translated;
-			
-			if (keys1 instanceof Keys.SelectorFunctionKeys && keys2 instanceof Keys.SelectorFunctionKeys) {
-				// Both join sides have a key selector function, so we need to do the
-				// tuple wrapping/unwrapping on both sides.
+					builder = builder
+							.withUdf(new TupleLeftUnwrappingJoiner<>(function))
+							.withWrappedInput1(input1, selectorKeys1, getInput1Type())
+							.withInput2(input2, getInput2Type(), keys2);
+				}
+			} else if (keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.ExpressionKeys) {
+				// Neither side needs the tuple wrapping/unwrapping
 
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
-				
-				PlanBothUnwrappingJoinOperator<I1, I2, OUT, ?> po =
-						translateSelectorFunctionJoin(selectorKeys1, selectorKeys2, function, 
-						getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
-				
-				// set parallelism
-				po.setParallelism(this.getParallelism());
-				
-				translated = po;
+				builder = builder
+						.withUdf(function)
+						.withInput1(input1, getInput1Type(), keys1)
+						.withInput2(input2, getInput2Type(), keys2);
+			} else {
+				throw new UnsupportedOperationException("Unrecognized or incompatible key types.");
 			}
-			else if (keys2 instanceof Keys.SelectorFunctionKeys) {
-				// The right side of the join needs the tuple wrapping/unwrapping
 
-				int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions();
+			return builder.build();
+		}
+
 
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 =
-						(Keys.SelectorFunctionKeys<I2, ?>) keys2;
+		private static final class JoinOperatorBaseBuilder<OUT> {
+
+			private final String name;
+			private final JoinType joinType;
 
-				PlanRightUnwrappingJoinOperator<I1, I2, OUT, ?> po =
-						translateSelectorFunctionJoinRight(logicalKeyPositions1, selectorKeys2,
-								function, getInput1Type(), getInput2Type(), getResultType(), name,
-								input1, input2);
+			private int parallelism;
+			private FlatJoinFunction<?, ?, OUT> udf;
+			private TypeInformation<OUT> resultType;
 
-				// set parallelism
-				po.setParallelism(this.getParallelism());
+			private Operator input1;
+			private TypeInformation<?> input1Type;
+			private Keys<?> keys1;
 
-				translated = po;
+			private Operator input2;
+			private TypeInformation<?> input2Type;
+			private Keys<?> keys2;
+
+			private Partitioner<?> partitioner;
+			private JoinHint joinHint;
+
+			public JoinOperatorBaseBuilder(String name, JoinType joinType) {
+				this.name = name;
+				this.joinType = joinType;
 			}
-			else if (keys1 instanceof Keys.SelectorFunctionKeys) {
-				// The left side of the join needs the tuple wrapping/unwrapping
 
+			public <I1, K> JoinOperatorBaseBuilder<OUT> withWrappedInput1(
+					Operator<I1> input1,
+					Keys.SelectorFunctionKeys<I1, ?> rawKeys1,
+					TypeInformation<I1> inputType1) {
+				TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<>(rawKeys1.getKeyType(), inputType1);
 
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 =
-						(Keys.SelectorFunctionKeys<I1, ?>) keys1;
+				MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
+						createKeyMapper(rawKeys1, inputType1, input1, "Key Extractor 1");
 
-				int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions();
+				return this.withInput1(keyMapper1, typeInfoWithKey1, rawKeys1);
+			}
 
-				PlanLeftUnwrappingJoinOperator<I1, I2, OUT, ?> po =
-						translateSelectorFunctionJoinLeft(selectorKeys1, logicalKeyPositions2, function,
-								getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
+			public <I2, K> JoinOperatorBaseBuilder<OUT> withWrappedInput2(
+					Operator<I2> input2,
+					Keys.SelectorFunctionKeys<I2, ?> rawKeys2,
+					TypeInformation<I2> inputType2) {
+				TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<>(rawKeys2.getKeyType(), inputType2);
 
-				// set parallelism
-				po.setParallelism(this.getParallelism());
+				MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
+						createKeyMapper(rawKeys2, inputType2, input2, "Key Extractor 2");
 
-				translated = po;
+				return withInput2(keyMapper2, typeInfoWithKey2, rawKeys2);
 			}
-			else if (super.keys1 instanceof Keys.ExpressionKeys && super.keys2 instanceof Keys.ExpressionKeys)
-			{
-				// Neither side needs the tuple wrapping/unwrapping
 
-				int[] logicalKeyPositions1 = super.keys1.computeLogicalKeyPositions();
-				int[] logicalKeyPositions2 = super.keys2.computeLogicalKeyPositions();
-				
-				JoinOperatorBase<I1, I2, OUT, FlatJoinFunction<I1, I2, OUT>> po =
-						new JoinOperatorBase<I1, I2, OUT, FlatJoinFunction<I1, I2, OUT>>(function,
-								new BinaryOperatorInformation<I1, I2, OUT>(getInput1Type(), getInput2Type(), getResultType()),
-								logicalKeyPositions1, logicalKeyPositions2,
-								name);
-				
-				// set inputs
-				po.setFirstInput(input1);
-				po.setSecondInput(input2);
-				// set parallelism
-				po.setParallelism(this.getParallelism());
-				
-				translated = po;
+			public <I1> JoinOperatorBaseBuilder<OUT> withInput1(
+					Operator<I1> input1,
+					TypeInformation<I1> input1Type,
+					Keys<?> keys1) {
+				this.input1 = input1;
+				this.input1Type = input1Type;
+				this.keys1 = keys1;
+				return this;
 			}
-			else {
-				throw new UnsupportedOperationException("Unrecognized or incompatible key types.");
+
+			public <I2> JoinOperatorBaseBuilder<OUT> withInput2(
+					Operator<I2> input2,
+					TypeInformation<I2> input2Type,
+					Keys<?> keys2) {
+				this.input2 = input2;
+				this.input2Type = input2Type;
+				this.keys2 = keys2;
+				return this;
 			}
-			
-			translated.setJoinHint(getJoinHint());
-			translated.setCustomPartitioner(getPartitioner());
-			
-			return translated;
-		}
-		
-		private static <I1, I2, K, OUT> PlanBothUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoin(
-				Keys.SelectorFunctionKeys<I1, ?> rawKeys1, Keys.SelectorFunctionKeys<I2, ?> rawKeys2, 
-				FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
-				Operator<I1> input1, Operator<I2> input2)
-		{
-			@SuppressWarnings("unchecked")
-			final Keys.SelectorFunctionKeys<I1, K> keys1 = (Keys.SelectorFunctionKeys<I1, K>) rawKeys1;
-			@SuppressWarnings("unchecked")
-			final Keys.SelectorFunctionKeys<I2, K> keys2 = (Keys.SelectorFunctionKeys<I2, K>) rawKeys2;
-			
-			final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1);
-			final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
-			
-			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>, 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 PlanBothUnwrappingJoinOperator<I1, I2, OUT, K> join = new PlanBothUnwrappingJoinOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
-			
-			join.setFirstInput(keyMapper1);
-			join.setSecondInput(keyMapper2);
-			
-			keyMapper1.setInput(input1);
-			keyMapper2.setInput(input2);
-			// set parallelism
-			keyMapper1.setParallelism(input1.getParallelism());
-			keyMapper2.setParallelism(input2.getParallelism());
-			
-			return join;
-		}
-		
-		private static <I1, I2, K, OUT> PlanRightUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinRight(
-				int[] logicalKeyPositions1,
-				Keys.SelectorFunctionKeys<I2, ?> rawKeys2,
-				FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<I1> inputType1,
-				TypeInformation<I2> inputType2,
-				TypeInformation<OUT> outputType,
-				String name,
-				Operator<I1> input1,
-				Operator<I2> input2) {
-
-			if(!inputType1.isTupleType()) {
-				throw new InvalidParameterException("Should not happen.");
+
+			public JoinOperatorBaseBuilder<OUT> withParallelism(int parallelism) {
+				this.parallelism = parallelism;
+				return this;
 			}
-			
-			@SuppressWarnings("unchecked")
-			final Keys.SelectorFunctionKeys<I2, K> keys2 =
-					(Keys.SelectorFunctionKeys<I2, K>) rawKeys2;
-			
-			final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 =
-					new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
-			
-			final KeyExtractingMapper<I2, K> extractor2 =
-					new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
-
-			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 PlanRightUnwrappingJoinOperator<I1, I2, OUT, K> join =
-					new PlanRightUnwrappingJoinOperator<I1, I2, OUT, K>(
-							function,
-							logicalKeyPositions1,
-							keys2,
-							name,
-							outputType,
-							inputType1,
-							typeInfoWithKey2);
-			
-			join.setFirstInput(input1);
-			join.setSecondInput(keyMapper2);
-			
-			keyMapper2.setInput(input2);
-			// set parallelism
-			keyMapper2.setParallelism(input2.getParallelism());
-			
-			return join;
-		}
-		
-		private static <I1, I2, K, OUT> PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinLeft(
-				Keys.SelectorFunctionKeys<I1, ?> rawKeys1,
-				int[] logicalKeyPositions2,
-				FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<I1> inputType1,
-				TypeInformation<I2> inputType2,
-				TypeInformation<OUT> outputType,
-				String name,
-				Operator<I1> input1,
-				Operator<I2> input2) {
-
-			if(!inputType2.isTupleType()) {
-				throw new InvalidParameterException("Should not happen.");
+
+			public JoinOperatorBaseBuilder<OUT> withPartitioner(Partitioner<?> partitioner) {
+				this.partitioner = partitioner;
+				return this;
 			}
-			
+
+			public JoinOperatorBaseBuilder<OUT> withJoinHint(JoinHint joinHint) {
+				this.joinHint = joinHint;
+				return this;
+			}
+
+			public JoinOperatorBaseBuilder<OUT> withUdf(FlatJoinFunction<?, ?, OUT> udf) {
+				this.udf = udf;
+				return this;
+			}
+
+			public JoinOperatorBaseBuilder<OUT> withResultType(TypeInformation<OUT> resultType) {
+				this.resultType = resultType;
+				return this;
+			}
+
 			@SuppressWarnings("unchecked")
-			final Keys.SelectorFunctionKeys<I1, K> keys1 = (Keys.SelectorFunctionKeys<I1, K>) rawKeys1;
-			
-			final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 =
-					new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1);
-
-			final KeyExtractingMapper<I1, K> extractor1 =
-					new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
-
-			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 PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K> join =
-					new PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K>(
-							function,
-							keys1,
-							logicalKeyPositions2,
-							name,
-							outputType,
-							typeInfoWithKey1,
-							inputType2);
-			
-			join.setFirstInput(keyMapper1);
-			join.setSecondInput(input2);
-			
-			keyMapper1.setInput(input1);
-			// set parallelism
-			keyMapper1.setParallelism(input1.getParallelism());
+			public JoinOperatorBase<?, ?, OUT, ?> build() {
+				JoinOperatorBase<?, ?, OUT, ?> operator;
+				if (joinType.isOuter()) {
+					operator = new OuterJoinOperatorBase<>(
+							udf,
+							new BinaryOperatorInformation(input1Type, input2Type, resultType),
+							this.keys1.computeLogicalKeyPositions(),
+							this.keys2.computeLogicalKeyPositions(),
+							this.name,
+							getOuterJoinType());
+				} else {
+					operator = new InnerJoinOperatorBase<>(
+							udf,
+							new BinaryOperatorInformation(input1Type, input2Type, resultType),
+							this.keys1.computeLogicalKeyPositions(),
+							this.keys2.computeLogicalKeyPositions(),
+							this.name);
+				}
+
+				operator.setFirstInput(input1);
+				operator.setSecondInput(input2);
+				operator.setParallelism(parallelism);
+				operator.setCustomPartitioner(partitioner);
+				operator.setJoinHint(joinHint);
+				return operator;
+			}
 
-			return join;
+			private OuterJoinOperatorBase.OuterJoinType getOuterJoinType() {
+				switch (joinType) {
+					case LEFT_OUTER:
+						return OuterJoinOperatorBase.OuterJoinType.LEFT;
+					case RIGHT_OUTER:
+						return OuterJoinOperatorBase.OuterJoinType.RIGHT;
+					case FULL_OUTER:
+						return OuterJoinOperatorBase.OuterJoinType.FULL;
+					default:
+						throw new UnsupportedOperationException();
+				}
+			}
+
+			private static <I, K> MapOperatorBase<I, Tuple2<K, I>, MapFunction<I, Tuple2<K, I>>> createKeyMapper(
+					Keys.SelectorFunctionKeys<I, ?> rawKeys,
+					TypeInformation<I> inputType,
+					Operator<I> input,
+					String mapperName) {
+
+				@SuppressWarnings("unchecked")
+				final Keys.SelectorFunctionKeys<I, K> keys = (Keys.SelectorFunctionKeys<I, K>) rawKeys;
+				final TypeInformation<Tuple2<K, I>> typeInfoWithKey = new TupleTypeInfo<>(keys.getKeyType(), inputType);
+				final KeyExtractingMapper<I, K> extractor = new KeyExtractingMapper<>(keys.getKeyExtractor());
+
+				final MapOperatorBase<I, Tuple2<K, I>, MapFunction<I, Tuple2<K, I>>> keyMapper =
+						new MapOperatorBase<I, Tuple2<K, I>, MapFunction<I, Tuple2<K, I>>>(
+								extractor,
+								new UnaryOperatorInformation<>(inputType, typeInfoWithKey),
+								mapperName);
+				keyMapper.setInput(input);
+				keyMapper.setParallelism(input.getParallelism());
+				return keyMapper;
+			}
 		}
 	}
 	
@@ -521,16 +532,16 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	 * @see Tuple2
 	 * @see DataSet
 	 */
-	public static final class DefaultJoin<I1, I2> extends EquiJoin<I1, I2, Tuple2<I1, I2>> {
+	public static final class DefaultJoin<I1, I2> extends EquiJoin<I1, I2, Tuple2<I1, I2>> implements JoinFunctionAssigner<I1, I2> {
 
-		protected DefaultJoin(DataSet<I1> input1, DataSet<I2> input2, 
-				Keys<I1> keys1, Keys<I2> keys2, JoinHint hint, String joinLocationName)
+		public DefaultJoin(DataSet<I1> input1, DataSet<I2> input2,
+				Keys<I1> keys1, Keys<I2> keys2, JoinHint hint, String joinLocationName, JoinType type)
 		{
-			super(input1, input2, keys1, keys2, 
-				(RichFlatJoinFunction<I1, I2, Tuple2<I1, I2>>) new DefaultFlatJoinFunction<I1, I2>(),
-				new TupleTypeInfo<Tuple2<I1, I2>>(input1.getType(), input2.getType()), hint, joinLocationName);
+			super(input1, input2, keys1, keys2,
+				new DefaultFlatJoinFunction<I1, I2>(),
+				new TupleTypeInfo<Tuple2<I1, I2>>(input1.getType(), input2.getType()), hint, joinLocationName, type);
 		}
-		
+
 		/**
 		 * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements.<br/>
 		 * Each JoinFunction call returns exactly one element. 
@@ -547,16 +558,16 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 				throw new NullPointerException("Join function must not be null.");
 			}
 			TypeInformation<R> returnType = TypeExtractor.getFlatJoinReturnTypes(function, getInput1Type(), getInput2Type());
-			return new EquiJoin<I1, I2, R>(getInput1(), getInput2(), getKeys1(), getKeys2(), clean(function), returnType, getJoinHint(), Utils.getCallLocationName());
+			return new EquiJoin<>(getInput1(), getInput2(), getKeys1(), getKeys2(), clean(function), returnType, getJoinHint(), Utils.getCallLocationName(), joinType);
 		}
 
 		public <R> EquiJoin<I1, I2, R> with (JoinFunction<I1, I2, R> function) {
 			if (function == null) {
 				throw new NullPointerException("Join function must not be null.");
 			}
-			FlatJoinFunction<I1, I2, R> generatedFunction = new WrappingFlatJoinFunction<I1, I2, R>(clean(function));
+			FlatJoinFunction<I1, I2, R> generatedFunction = new WrappingFlatJoinFunction<>(clean(function));
 			TypeInformation<R> returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type());
-			return new EquiJoin<I1, I2, R>(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint(), Utils.getCallLocationName());
+			return new EquiJoin<>(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint(), Utils.getCallLocationName(), joinType);
 		}
 
 		public static class WrappingFlatJoinFunction<IN1, IN2, OUT> extends WrappingFunction<JoinFunction<IN1,IN2,OUT>> implements FlatJoinFunction<IN1, IN2, OUT> {
@@ -582,7 +593,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}.
 		 *
-		 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
+		 * <b>Note: With the current implementation, the Project transformation loses type information.</b>
 		 *
 		 * @param firstFieldIndexes If the first input is a Tuple DataSet, the indexes of the selected fields.
 		 * 					   For a non-Tuple DataSet, do not provide parameters.
@@ -595,7 +606,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 */
 		public <OUT extends Tuple> ProjectJoin<I1, I2, OUT> projectFirst(int... firstFieldIndexes) {
 			JoinProjection<I1, I2> joinProjection = new JoinProjection<I1, I2>(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint(), firstFieldIndexes, null);
-			
+
 			return joinProjection.projectTupleX();
 		}
 		
@@ -608,7 +619,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}.
 		 *
-		 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
+		 * <b>Note: With the current implementation, the Project transformation loses type information.</b>
 		 *
 		 * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. 
 		 * 					   For a non-Tuple DataSet, do not provide parameters.
@@ -624,7 +635,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			
 			return joinProjection.projectTupleX();
 		}
-		
+
 //		public JoinOperator<I1, I2, I1> leftSemiJoin() {
 //			return new LeftSemiJoin<I1, I2>(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint());
 //		}
@@ -659,22 +670,21 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		private JoinProjection<I1, I2> joinProj;
 		
 		protected ProjectJoin(DataSet<I1> input1, DataSet<I2> input2, Keys<I1> keys1, Keys<I2> keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo<OUT> returnType) {
-			super(input1, input2, keys1, keys2, 
+			super(input1, input2, keys1, keys2,
 					new ProjectFlatJoinFunction<I1, I2, OUT>(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()),
 					returnType, hint, Utils.getCallLocationName(4)); // We need to use the 4th element in the stack because the call comes through .types().
 
-			
 			joinProj = null;
 		}
 		
 		protected ProjectJoin(DataSet<I1> input1, DataSet<I2> input2, Keys<I1> keys1, Keys<I2> keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo<OUT> returnType, JoinProjection<I1, I2> joinProj) {
-			super(input1, input2, keys1, keys2, 
+			super(input1, input2, keys1, keys2,
 					new ProjectFlatJoinFunction<I1, I2, OUT>(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()),
 					returnType, hint, Utils.getCallLocationName(4));
-			
+
 			this.joinProj = joinProj;
 		}
-		
+
 		@Override
 		protected ProjectFlatJoinFunction<I1, I2, OUT> getFunction() {
 			return (ProjectFlatJoinFunction<I1, I2, OUT>) super.getFunction();
@@ -689,7 +699,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}.
 		 *
-		 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
+		 * <b>Note: With the current implementation, the Project transformation loses type information.</b>
 		 *
 		 * @param firstFieldIndexes If the first input is a Tuple DataSet, the indexes of the selected fields.
 		 * 					   For a non-Tuple DataSet, do not provide parameters.
@@ -716,7 +726,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}.
 		 *
-		 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
+		 * <b>Note: With the current implementation, the Project transformation loses type information.</b>
 		 *
 		 * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields.
 		 * 					   For a non-Tuple DataSet, do not provide parameters.
@@ -834,188 +844,6 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 //	}
 	
 	// --------------------------------------------------------------------------------------------
-	// Builder classes for incremental construction
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Intermediate step of a Join transformation. <br/>
-	 * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling 
-	 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets#where(int...)} or
-	 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets#where(KeySelector)}.
-	 *
-	 * @param <I1> The type of the first input DataSet of the Join transformation.
-	 * @param <I2> The type of the second input DataSet of the Join transformation.
-	 */
-	public static final class JoinOperatorSets<I1, I2> {
-		
-		private final DataSet<I1> input1;
-		private final DataSet<I2> input2;
-		
-		private final JoinHint joinHint;
-		
-		public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
-			this(input1, input2, JoinHint.OPTIMIZER_CHOOSES);
-		}
-		
-		public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
-			if (input1 == null || input2 == null) {
-				throw new NullPointerException();
-			}
-			
-			this.input1 = input1;
-			this.input2 = input2;
-			this.joinHint = hint;
-		}
-		
-		/**
-		 * Continues a Join transformation. <br/>
-		 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br/>
-		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-		 *
-		 * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys.
-		 * @return An incomplete Join transformation. 
-		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
-		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
-		 *           to continue the Join. 
-		 * 
-		 * @see Tuple
-		 * @see DataSet
-		 */
-		public JoinOperatorSetsPredicate where(int... fields) {
-			return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<I1>(fields, input1.getType()));
-		}
-
-		/**
-		 * Continues a Join transformation. <br/>
-		 * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields
-		 * are the names of member fields of the underlying type of the data set.
-		 *
-		 * @param fields The  fields of the first join DataSets that should be used as keys.
-		 * @return An incomplete Join transformation.
-		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
-		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
-		 *           to continue the Join.
-		 *
-		 * @see Tuple
-		 * @see DataSet
-		 */
-		public JoinOperatorSetsPredicate where(String... fields) {
-			return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<I1>(fields, input1.getType()));
-		}
-		
-		/**
-		 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.</br>
-		 * The KeySelector function is called for each element of the first DataSet and extracts a single 
-		 * key value on which the DataSet is joined. </br>
-		 * 
-		 * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined.
-		 * @return An incomplete Join transformation. 
-		 *           Call {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
-		 *           {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}
-		 *           to continue the Join. 
-		 * 
-		 * @see KeySelector
-		 * @see DataSet
-		 */
-		public <K> JoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
-			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
-			return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<I1, K>(keySelector, input1.getType(), keyType));
-		}
-		
-		// ----------------------------------------------------------------------------------------
-		
-		/**
-		 * Intermediate step of a Join transformation. <br/>
-		 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling 
-		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
-		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}.
-		 *
-		 */
-		public class JoinOperatorSetsPredicate {
-			
-			private final Keys<I1> keys1;
-			
-			private JoinOperatorSetsPredicate(Keys<I1> keys1) {
-				if (keys1 == null) {
-					throw new NullPointerException();
-				}
-				
-				if (keys1.isEmpty()) {
-					throw new InvalidProgramException("The join keys must not be empty.");
-				}
-				
-				this.keys1 = keys1;
-			}
-			
-			/**
-			 * Continues a Join transformation and defines the {@link Tuple} fields of the second join 
-			 * {@link DataSet} that should be used as join keys.<br/>
-			 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-			 * 
-			 * The resulting {@link org.apache.flink.api.java.operators.JoinOperator.DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with 
-			 * the element of the first input being the first field of the tuple and the element of the 
-			 * second input being the second field of the tuple. 
-			 *
-			 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
-			 * @return A DefaultJoin that represents the joined DataSet.
-			 */
-			public DefaultJoin<I1, I2> equalTo(int... fields) {
-				return createJoinOperator(new Keys.ExpressionKeys<I2>(fields, input2.getType()));
-			}
-
-			/**
-			 * Continues a Join transformation and defines the  fields of the second join
-			 * {@link DataSet} that should be used as join keys.<br/>
-			 *
-			 * The resulting {@link org.apache.flink.api.java.operators.JoinOperator.DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
-			 * the element of the first input being the first field of the tuple and the element of the
-			 * second input being the second field of the tuple.
-			 *
-			 * @param fields The fields of the second join DataSet that should be used as keys.
-			 * @return A DefaultJoin that represents the joined DataSet.
-			 */
-			public DefaultJoin<I1, I2> equalTo(String... fields) {
-				return createJoinOperator(new Keys.ExpressionKeys<I2>(fields, input2.getType()));
-			}
-
-			/**
-			 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
-			 * The KeySelector function is called for each element of the second DataSet and extracts a single 
-			 * key value on which the DataSet is joined. </br>
-			 * 
-			 * The resulting {@link org.apache.flink.api.java.operators.JoinOperator.DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with 
-			 * the element of the first input being the first field of the tuple and the element of the 
-			 * second input being the second field of the tuple. 
-			 * 
-			 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
-			 * @return A DefaultJoin that represents the joined DataSet.
-			 */
-			public <K> DefaultJoin<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
-				TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
-				return createJoinOperator(new Keys.SelectorFunctionKeys<I2, K>(keySelector, input2.getType(), keyType));
-			}
-			
-			protected DefaultJoin<I1, I2> createJoinOperator(Keys<I2> keys2) {
-				if (keys2 == null) {
-					throw new NullPointerException("The join keys may not be null.");
-				}
-				
-				if (keys2.isEmpty()) {
-					throw new InvalidProgramException("The join keys may not be empty.");
-				}
-				
-				try {
-					keys1.areCompatible(keys2);
-				} catch (IncompatibleKeysException e) {
-					throw new InvalidProgramException("The pair of join keys are not compatible with each other.",e);
-				}
-
-				return new DefaultJoin<I1, I2>(input1, input2, keys1, keys2, joinHint, Utils.getCallLocationName(4));
-			}
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
 	//  default join functions
 	// --------------------------------------------------------------------------------------------
 
@@ -1033,7 +861,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			out.collect(outTuple);
 		}
 	}
-	
+
 	public static final class ProjectFlatJoinFunction<T1, T2, R extends Tuple> extends RichFlatJoinFunction<T1, T2, R> {
 		
 		private static final long serialVersionUID = 1L;
@@ -1052,10 +880,10 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * @param outTupleInstance An instance of an output tuple.
 		 */
 		private ProjectFlatJoinFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) {
-			
 			if(fields.length != isFromFirst.length) {
 				throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); 
 			}
+
 			this.fields = fields;
 			this.isFromFirst = isFromFirst;
 			this.outTuple = outTupleInstance;
@@ -1070,16 +898,16 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 
 		public void join(T1 in1, T2 in2, Collector<R> out) {
-			for(int i=0; i<fields.length; i++) {
-				if(isFromFirst[i]) {
-					if(fields[i] >= 0 && in1 != null) {
-						outTuple.setField(((Tuple)in1).getField(fields[i]), i);
+			for (int i = 0; i < fields.length; i++) {
+				if (isFromFirst[i]) {
+					if (fields[i] >= 0 && in1 != null) {
+						outTuple.setField(((Tuple) in1).getField(fields[i]), i);
 					} else {
 						outTuple.setField(in1, i);
 					}
 				} else {
-					if(fields[i] >= 0 && in2 != null) {
-						outTuple.setField(((Tuple)in2).getField(fields[i]), i);
+					if (fields[i] >= 0 && in2 != null) {
+						outTuple.setField(((Tuple) in2).getField(fields[i]), i);
 					} else {
 						outTuple.setField(in2, i);
 					}
@@ -1097,7 +925,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		private final Keys<I1> keys1;
 		private final Keys<I2> keys2;
 		private final JoinHint hint;
-		
+
 		private int[] fieldIndexes;
 		private boolean[] isFieldInFirst;
 		
@@ -1105,13 +933,12 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		private final int numFieldsDs2;
 		
 		public JoinProjection(DataSet<I1> ds1, DataSet<I2> ds2, Keys<I1> keys1, Keys<I2> keys2, JoinHint hint, int[] firstFieldIndexes, int[] secondFieldIndexes) {
-			
 			this.ds1 = ds1;
 			this.ds2 = ds2;
 			this.keys1 = keys1;
 			this.keys2 = keys2;
 			this.hint = hint;
-			
+
 			boolean isFirstTuple;
 			boolean isSecondTuple;
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
new file mode 100644
index 0000000..e0e15ca
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/InnerJoinOperatorSets.java
@@ -0,0 +1,154 @@
+/*
+ * 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.join;
+
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+/**
+ * Intermediate step of a Join transformation. <br/>
+ * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
+ * {@link InnerJoinOperatorSets#where(int...)} or
+ * {@link InnerJoinOperatorSets#where(KeySelector)}.
+ *
+ * @param <I1> The type of the first input DataSet of the Join transformation.
+ * @param <I2> The type of the second input DataSet of the Join transformation.
+ */
+public final class InnerJoinOperatorSets<I1, I2> extends JoinOperatorSets<I1, I2> {
+
+	public InnerJoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
+		super(input1, input2);
+	}
+
+	public InnerJoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
+		super(input1, input2, hint);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 *
+	 * @return An incomplete Join transformation.
+	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 */
+	@Override
+	public InnerJoinOperatorSetsPredicate where(int... fields) {
+		return new InnerJoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * {@inheritDoc}
+	 *
+	 * @return An incomplete Join transformation.
+	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 */
+	@Override
+	public InnerJoinOperatorSetsPredicate where(String... fields) {
+		return new InnerJoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * {@inheritDoc}
+	 *
+	 * @return An incomplete Join transformation.
+	 *           Call {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 */
+	@Override
+	public <K> InnerJoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
+		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+		return new InnerJoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
+	}
+
+
+	/**
+	 * Intermediate step of a Join transformation. <br/>
+	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
+	 * {@link InnerJoinOperatorSetsPredicate#equalTo(int...)} or
+	 * {@link InnerJoinOperatorSetsPredicate#equalTo(KeySelector)}.
+	 */
+	public class InnerJoinOperatorSetsPredicate extends JoinOperatorSetsPredicate {
+
+		private InnerJoinOperatorSetsPredicate(Keys<I1> keys1) {
+			super(keys1);
+		}
+
+		/**
+		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+		 * <p/>
+		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+		 * the element of the first input being the first field of the tuple and the element of the
+		 * second input being the second field of the tuple.
+		 *
+		 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
+		 * @return A DefaultJoin that represents the joined DataSet.
+		 */
+		@Override
+		public DefaultJoin<I1, I2> equalTo(int... fields) {
+			return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines the fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 * <p/>
+		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+		 * the element of the first input being the first field of the tuple and the element of the
+		 * second input being the second field of the tuple.
+		 *
+		 * @param fields The fields of the second join DataSet that should be used as keys.
+		 * @return A DefaultJoin that represents the joined DataSet.
+		 */
+		@Override
+		public DefaultJoin<I1, I2> equalTo(String... fields) {
+			return createDefaultJoin(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+		 * The KeySelector function is called for each element of the second DataSet and extracts a single
+		 * key value on which the DataSet is joined. </br>
+		 * <p/>
+		 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
+		 * the element of the first input being the first field of the tuple and the element of the
+		 * second input being the second field of the tuple.
+		 *
+		 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
+		 * @return A DefaultJoin that represents the joined DataSet.
+		 */
+		@Override
+		public <K> DefaultJoin<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+			return createDefaultJoin(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinFunctionAssigner.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinFunctionAssigner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinFunctionAssigner.java
new file mode 100644
index 0000000..163c5a6
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinFunctionAssigner.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.join;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.operators.JoinOperator;
+
+/**
+ * A Join transformation that needs to be finished by specifying either a
+ * {@link JoinFunction} or a {@link FlatJoinFunction} before it can be used as an input
+ * to other operators.
+ *
+ * @param <I1> The type of the first input DataSet of the Join transformation.
+ * @param <I2> The type of the second input DataSet of the Join transformation.
+ */
+public interface JoinFunctionAssigner<I1, I2> {
+
+	<R> JoinOperator<I1, I2, R> with(JoinFunction<I1, I2, R> joinFunction);
+
+	<R> JoinOperator<I1, I2, R> with(FlatJoinFunction<I1, I2, R> joinFunction);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
new file mode 100644
index 0000000..705952c
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSets.java
@@ -0,0 +1,235 @@
+/*
+ * 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.join;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
+import org.apache.flink.api.java.operators.JoinOperator.EquiJoin;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+/**
+ * Intermediate step of an Outer Join transformation. <br/>
+ * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
+ * {@link JoinOperatorSets#where(int...)} or
+ * {@link JoinOperatorSets#where(KeySelector)}.
+ *
+ * @param <I1> The type of the first input DataSet of the Join transformation.
+ * @param <I2> The type of the second input DataSet of the Join transformation.
+ */
+public class JoinOperatorSets<I1, I2> {
+
+	protected final DataSet<I1> input1;
+	protected final DataSet<I2> input2;
+
+	protected final JoinHint joinHint;
+	protected final JoinType joinType;
+
+	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2) {
+		this(input1, input2, JoinHint.OPTIMIZER_CHOOSES);
+	}
+
+	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint) {
+		this(input1, input2, hint, JoinType.INNER);
+	}
+
+	public JoinOperatorSets(DataSet<I1> input1, DataSet<I2> input2, JoinHint hint, JoinType type) {
+		if (input1 == null || input2 == null) {
+			throw new NullPointerException();
+		}
+
+		this.input1 = input1;
+		this.input2 = input2;
+		this.joinHint = hint;
+		this.joinType = type;
+	}
+
+	/**
+	 * Continues a Join transformation. <br/>
+	 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br/>
+	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+	 *
+	 * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see Tuple
+	 * @see DataSet
+	 */
+	public JoinOperatorSetsPredicate where(int... fields) {
+		return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * Continues a Join transformation. <br/>
+	 * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields
+	 * are the names of member fields of the underlying type of the data set.
+	 *
+	 * @param fields The  fields of the first join DataSets that should be used as keys.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see Tuple
+	 * @see DataSet
+	 */
+	public JoinOperatorSetsPredicate where(String... fields) {
+		return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys<>(fields, input1.getType()));
+	}
+
+	/**
+	 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.</br>
+	 * The KeySelector function is called for each element of the first DataSet and extracts a single
+	 * key value on which the DataSet is joined. </br>
+	 *
+	 * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined.
+	 * @return An incomplete Join transformation.
+	 *           Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or
+	 *           {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}
+	 *           to continue the Join.
+	 *
+	 * @see KeySelector
+	 * @see DataSet
+	 */
+	public <K> JoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
+		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+		return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType));
+	}
+
+
+	/**
+	 * Intermediate step of a Join transformation. <br/>
+	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
+	 * {@link JoinOperatorSetsPredicate#equalTo(int...)} or
+	 * {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}.
+	 *
+	 */
+	public class JoinOperatorSetsPredicate {
+
+		protected final Keys<I1> keys1;
+
+		protected JoinOperatorSetsPredicate(Keys<I1> keys1) {
+			if (keys1 == null) {
+				throw new NullPointerException();
+			}
+
+			if (keys1.isEmpty()) {
+				throw new InvalidProgramException("The join keys must not be empty.");
+			}
+
+			this.keys1 = keys1;
+		}
+
+		/**
+		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public JoinFunctionAssigner<I1, I2> equalTo(int... fields) {
+			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines the fields of the second join
+		 * {@link DataSet} that should be used as join keys.<br/>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param fields The fields of the second join DataSet that should be used as keys.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public JoinFunctionAssigner<I1, I2> equalTo(String... fields) {
+			return createJoinFunctionAssigner(new Keys.ExpressionKeys<>(fields, input2.getType()));
+		}
+
+		/**
+		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+		 * The KeySelector function is called for each element of the second DataSet and extracts a single
+		 * key value on which the DataSet is joined. </br>
+		 *
+		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
+		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
+		 *
+		 * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined.
+		 * @return A JoinFunctionAssigner.
+		 */
+		public <K> JoinFunctionAssigner<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+			return createJoinFunctionAssigner(new Keys.SelectorFunctionKeys<>(keySelector, input2.getType(), keyType));
+		}
+
+		protected JoinFunctionAssigner<I1, I2> createJoinFunctionAssigner(Keys<I2> keys2) {
+			DefaultJoin<I1, I2> join = createDefaultJoin(keys2);
+			return new DefaultJoinFunctionAssigner(join);
+		}
+
+		protected DefaultJoin<I1, I2> createDefaultJoin(Keys<I2> keys2) {
+			if (keys2 == null) {
+				throw new NullPointerException("The join keys may not be null.");
+			}
+
+			if (keys2.isEmpty()) {
+				throw new InvalidProgramException("The join keys may not be empty.");
+			}
+
+			try {
+				keys1.areCompatible(keys2);
+			} catch (Keys.IncompatibleKeysException e) {
+				throw new InvalidProgramException("The pair of join keys are not compatible with each other.",e);
+			}
+			return new DefaultJoin<>(input1, input2, keys1, keys2, joinHint, Utils.getCallLocationName(4), joinType);
+		}
+
+		private class DefaultJoinFunctionAssigner implements JoinFunctionAssigner<I1, I2> {
+
+			private final DefaultJoin<I1, I2> defaultJoin;
+
+			public DefaultJoinFunctionAssigner(DefaultJoin<I1, I2> defaultJoin) {
+				this.defaultJoin = defaultJoin;
+			}
+
+			public <R> EquiJoin<I1, I2, R> with(JoinFunction<I1, I2, R> joinFunction) {
+				return defaultJoin.with(joinFunction);
+			}
+
+			public <R> EquiJoin<I1, I2, R> with(FlatJoinFunction<I1, I2, R> joinFunction) {
+				return defaultJoin.with(joinFunction);
+			}
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java
new file mode 100644
index 0000000..9d00fab
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.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.java.operators.join;
+
+public enum JoinType {
+
+	INNER, LEFT_OUTER, RIGHT_OUTER, FULL_OUTER;
+
+	public boolean isOuter() {
+		return this == LEFT_OUTER || this == RIGHT_OUTER || this == FULL_OUTER;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
deleted file mode 100644
index e9ded4f..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
+++ /dev/null
@@ -1,66 +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.operators.translation;
-
-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.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class PlanBothUnwrappingJoinOperator<I1, I2, OUT, K>
-	extends JoinOperatorBase<Tuple2<K, I1>, Tuple2<K, I2>, OUT, FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>>
-{
-
-	public PlanBothUnwrappingJoinOperator(
-			FlatJoinFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1,
-			Keys.SelectorFunctionKeys<I2, K> key2, String name,
-			TypeInformation<OUT> resultType,
-			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
-			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
-
-		super(
-				new TupleUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(
-						typeInfoWithKey1,
-						typeInfoWithKey2,
-						resultType),
-				key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name);
-	}
-
-	public static final class TupleUnwrappingJoiner<I1, I2, OUT, K>
-		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
-		implements FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT> {
-
-		private static final long serialVersionUID = 1L;
-		
-		private TupleUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
-			super(wrapped);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void join (Tuple2<K, I1> value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
-			wrappedFunction.join ((I1)(value1.getField(1)), (I2)(value2.getField(1)), collector);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
deleted file mode 100644
index c6ee804..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
+++ /dev/null
@@ -1,64 +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.operators.translation;
-
-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.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K>
-		extends JoinOperatorBase<Tuple2<K, I1>, I2, OUT, FlatJoinFunction<Tuple2<K, I1>, I2, OUT>> {
-
-	public PlanLeftUnwrappingJoinOperator(
-			FlatJoinFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1,
-			int[] key2, String name,
-			TypeInformation<OUT> resultType,
-			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
-			TypeInformation<I2> typeInfo2) {
-		super(
-				new TupleLeftUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, I2, OUT>(
-						typeInfoWithKey1,
-						typeInfo2,
-						resultType),
-				key1.computeLogicalKeyPositions(), key2, name);
-	}
-
-	public static final class TupleLeftUnwrappingJoiner<I1, I2, OUT, K>
-			extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
-			implements FlatJoinFunction<Tuple2<K, I1>, I2, OUT> {
-
-		private static final long serialVersionUID = 1L;
-
-		private TupleLeftUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
-			super(wrapped);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void join (Tuple2<K, I1> value1, I2 value2, Collector<OUT> collector) throws Exception {
-			wrappedFunction.join ((I1)(value1.getField(1)), value2, collector);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
deleted file mode 100644
index dc460f8..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
+++ /dev/null
@@ -1,66 +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.operators.translation;
-
-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.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class PlanRightUnwrappingJoinOperator<I1, I2, OUT, K>
-		extends JoinOperatorBase<I1, Tuple2<K, I2>, OUT, FlatJoinFunction<I1, Tuple2<K, I2>, OUT>> {
-
-	public PlanRightUnwrappingJoinOperator(
-			FlatJoinFunction<I1, I2, OUT> udf,
-			int[] key1,
-			Keys.SelectorFunctionKeys<I2, K> key2,
-			String name,
-			TypeInformation<OUT> type,
-			TypeInformation<I1> typeInfo1,
-			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
-
-		super(
-				new TupleRightUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<I1, Tuple2<K, I2>, OUT>(
-						typeInfo1,
-						typeInfoWithKey2,
-						type),
-				key1, key2.computeLogicalKeyPositions(), name);
-	}
-
-	public static final class TupleRightUnwrappingJoiner<I1, I2, OUT, K>
-			extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
-			implements FlatJoinFunction<I1, Tuple2<K, I2>, OUT> {
-
-		private static final long serialVersionUID = 1L;
-
-		private TupleRightUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
-			super(wrapped);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void join (I1 value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
-			wrappedFunction.join (value1, (I2)(value2.getField(1)), collector);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java
new file mode 100644
index 0000000..18d3378
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java
@@ -0,0 +1,40 @@
+/*
+ * 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.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+public final class TupleLeftUnwrappingJoiner<I1, I2, OUT, K>
+		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+		implements FlatJoinFunction<Tuple2<K, I1>, I2, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	public TupleLeftUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+		super(wrapped);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void join(Tuple2<K, I1> value1, I2 value2, Collector<OUT> collector) throws Exception {
+		wrappedFunction.join(value1 == null ? null : (I1) value1.getField(1), value2, collector);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java
new file mode 100644
index 0000000..004a0ae
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java
@@ -0,0 +1,40 @@
+/*
+ * 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.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+public final class TupleRightUnwrappingJoiner<I1, I2, OUT, K>
+		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+		implements FlatJoinFunction<I1, Tuple2<K, I2>, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	public TupleRightUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+		super(wrapped);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void join(I1 value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
+		wrappedFunction.join(value1, value2 == null ? null : (I2) value2.getField(1), collector);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java
new file mode 100644
index 0000000..15cc137
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+public final class TupleUnwrappingJoiner<I1, I2, OUT, K>
+		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+		implements FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	public TupleUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+		super(wrapped);
+	}
+
+	@Override
+	public void join(Tuple2<K, I1> value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
+		wrappedFunction.join(unwrap(value1), unwrap(value2), collector);
+	}
+
+	@SuppressWarnings("unchecked")
+	private <V> V unwrap(Tuple2<K, V> t) {
+		return t == null ? null : (V) (t.getField(1));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/record/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/JoinOperator.java
index 6dc150e..75744fe 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/JoinOperator.java
@@ -26,7 +26,7 @@ import java.util.Map;
 
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.RecordOperator;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
 import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
 import org.apache.flink.api.common.operators.util.UserCodeWrapper;
@@ -50,7 +50,7 @@ import com.google.common.base.Preconditions;
  */
 
 @Deprecated
-public class JoinOperator extends JoinOperatorBase<Record, Record, Record, JoinFunction> implements RecordOperator {
+public class JoinOperator extends InnerJoinOperatorBase<Record, Record, Record, JoinFunction> implements RecordOperator {
 	
 	/**
 	 * The types of the keys that the operator operates on.

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
new file mode 100644
index 0000000..8bc29d4
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.operators.base;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Future;
+
+@SuppressWarnings({ "unchecked", "serial" })
+public class InnerJoinOperatorBaseTest implements Serializable {
+
+	
+	@Test
+	public void testTupleBaseJoiner(){
+		final FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double, String>> joiner =
+					new FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double, String>>()
+		{
+			@Override
+			public void join(Tuple3<String, Double, Integer> first, Tuple2<Integer, String> second, Collector<Tuple2<Double, String>> out) {
+
+				assertEquals(first.f0, second.f1);
+				assertEquals(first.f2, second.f0);
+
+				out.collect(new Tuple2<>(first.f1, second.f0.toString()));
+			}
+		};
+
+		final TupleTypeInfo<Tuple3<String, Double, Integer>> leftTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo
+				(String.class, Double.class, Integer.class);
+		final TupleTypeInfo<Tuple2<Integer, String>> rightTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo(Integer.class,
+				String.class);
+		final TupleTypeInfo<Tuple2<Double, String>> outTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo(Double.class,
+				String.class);
+
+		final int[] leftKeys = new int[]{0,2};
+		final int[] rightKeys = new int[]{1,0};
+
+		final String taskName = "Collection based tuple joiner";
+
+		final BinaryOperatorInformation<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double,
+				String>> binaryOpInfo = new BinaryOperatorInformation<Tuple3<String, Double, Integer>, Tuple2<Integer,
+				String>, Tuple2<Double, String>>(leftTypeInfo, rightTypeInfo, outTypeInfo);
+
+		final InnerJoinOperatorBase<Tuple3<String, Double, Integer>, Tuple2<Integer,
+						String>, Tuple2<Double, String>, FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer,
+						String>, Tuple2<Double, String>>> base = new InnerJoinOperatorBase<Tuple3<String, Double, Integer>,
+										Tuple2<Integer, String>, Tuple2<Double, String>, FlatJoinFunction<Tuple3<String, Double, Integer>,
+										Tuple2<Integer, String>, Tuple2<Double, String>>>(joiner, binaryOpInfo, leftKeys, rightKeys, taskName);
+
+		final List<Tuple3<String, Double, Integer> > inputData1 = new ArrayList<Tuple3<String, Double,
+				Integer>>(Arrays.asList(
+				new Tuple3<>("foo", 42.0, 1),
+				new Tuple3<>("bar", 1.0, 2),
+				new Tuple3<>("bar", 2.0, 3),
+				new Tuple3<>("foobar", 3.0, 4),
+				new Tuple3<>("bar", 3.0, 3)
+		));
+
+		final List<Tuple2<Integer, String>> inputData2 = new ArrayList<Tuple2<Integer, String>>(Arrays.asList(
+				new Tuple2<>(3, "bar"),
+				new Tuple2<>(4, "foobar"),
+				new Tuple2<>(2, "foo")
+		));
+		final Set<Tuple2<Double, String>> expected = new HashSet<Tuple2<Double, String>>(Arrays.asList(
+				new Tuple2<>(2.0, "3"),
+				new Tuple2<>(3.0, "3"),
+				new Tuple2<>(3.0, "4")
+		));
+
+		try {
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			executionConfig.disableObjectReuse();
+			List<Tuple2<Double, String>> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig, new HashMap<String, Future<Path>>(), new HashMap<String, Accumulator<?, ?>>()), executionConfig);
+			executionConfig.enableObjectReuse();
+			List<Tuple2<Double, String>> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig, new HashMap<String, Future<Path>>(), new HashMap<String, Accumulator<?, ?>>()), executionConfig);
+
+			assertEquals(expected, new HashSet<>(resultSafe));
+			assertEquals(expected, new HashSet<>(resultRegular));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[6/6] flink git commit: [FLINK-2576] Add outer join base operator.

Posted by fh...@apache.org.
[FLINK-2576] Add outer join base operator.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6b222276
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6b222276
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6b222276

Branch: refs/heads/master
Commit: 6b2222762fc38d84b31170216d6b6ae0c272af9b
Parents: 0455857
Author: r-pogalz <r....@campus.tu-berlin.de>
Authored: Tue Jul 7 21:40:04 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Oct 9 16:19:21 2015 +0200

----------------------------------------------------------------------
 .../operators/base/OuterJoinOperatorBase.java   | 314 +++++++++++++++++++
 .../base/OuterJoinOperatorBaseTest.java         | 150 +++++++++
 .../runtime/operators/FullOuterJoinDriver.java  |   2 +-
 .../runtime/operators/LeftOuterJoinDriver.java  |   2 +-
 .../runtime/operators/RightOuterJoinDriver.java |   2 +-
 .../sort/AbstractMergeOuterJoinIterator.java    |   3 +-
 .../sort/NonReusingMergeOuterJoinIterator.java  |   1 +
 .../sort/ReusingMergeOuterJoinIterator.java     |   1 +
 ...bstractSortMergeOuterJoinIteratorITCase.java |   2 +-
 ...ReusingSortMergeOuterJoinIteratorITCase.java |   2 +-
 ...ReusingSortMergeOuterJoinIteratorITCase.java |   2 +-
 11 files changed, 473 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
new file mode 100644
index 0000000..7666d10
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
@@ -0,0 +1,314 @@
+/*
+ * 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.operators.base;
+
+import org.apache.commons.collections.ResettableIterator;
+import org.apache.commons.collections.iterators.ListIteratorWrapper;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.functions.util.CopyingListCollector;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.util.ListKeyGroupedIterator;
+import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
+import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.CompositeType;
+import org.apache.flink.api.common.typeutils.GenericPairComparator;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+public class OuterJoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends AbstractJoinOperatorBase<IN1, IN2, OUT, FT> {
+
+	public static enum OuterJoinType {LEFT, RIGHT, FULL}
+
+	private OuterJoinType outerJoinType;
+
+	public OuterJoinOperatorBase(UserCodeWrapper<FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo,
+			int[] keyPositions1, int[] keyPositions2, String name, OuterJoinType outerJoinType) {
+		super(udf, operatorInfo, keyPositions1, keyPositions2, name);
+		this.outerJoinType = outerJoinType;
+	}
+
+	public OuterJoinOperatorBase(FT udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo,
+			int[] keyPositions1, int[] keyPositions2, String name, OuterJoinType outerJoinType) {
+		super(new UserCodeObjectWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+		this.outerJoinType = outerJoinType;
+	}
+
+	public OuterJoinOperatorBase(Class<? extends FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo,
+			int[] keyPositions1, int[] keyPositions2, String name, OuterJoinType outerJoinType) {
+		super(new UserCodeClassWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+		this.outerJoinType = outerJoinType;
+	}
+
+	public void setOuterJoinType(OuterJoinType outerJoinType) {
+		this.outerJoinType = outerJoinType;
+	}
+
+	public OuterJoinType getOuterJoinType() {
+		return outerJoinType;
+	}
+
+	@Override
+	protected List<OUT> executeOnCollections(List<IN1> leftInput, List<IN2> rightInput, RuntimeContext runtimeContext, ExecutionConfig executionConfig) throws Exception {
+		TypeInformation<IN1> leftInformation = getOperatorInfo().getFirstInputType();
+		TypeInformation<IN2> rightInformation = getOperatorInfo().getSecondInputType();
+		TypeInformation<OUT> outInformation = getOperatorInfo().getOutputType();
+
+		TypeComparator<IN1> leftComparator = buildComparatorFor(0, executionConfig, leftInformation);
+		TypeComparator<IN2> rightComparator = buildComparatorFor(1, executionConfig, rightInformation);
+
+		TypeSerializer<IN1> leftSerializer = leftInformation.createSerializer(executionConfig);
+		TypeSerializer<IN2> rightSerializer = rightInformation.createSerializer(executionConfig);
+
+		OuterJoinListIterator<IN1, IN2> outerJoinIterator =
+				new OuterJoinListIterator<>(leftInput, leftSerializer, leftComparator,
+						rightInput, rightSerializer, rightComparator, outerJoinType);
+
+		// --------------------------------------------------------------------
+		// Run UDF
+		// --------------------------------------------------------------------
+		FlatJoinFunction<IN1, IN2, OUT> function = userFunction.getUserCodeObject();
+
+		FunctionUtils.setFunctionRuntimeContext(function, runtimeContext);
+		FunctionUtils.openFunction(function, this.parameters);
+
+
+		List<OUT> result = new ArrayList<>();
+		Collector<OUT> collector = new CopyingListCollector<>(result, outInformation.createSerializer(executionConfig));
+
+		while (outerJoinIterator.next()) {
+			IN1 left = outerJoinIterator.getLeft();
+			IN2 right = outerJoinIterator.getRight();
+			function.join(left == null ? null : leftSerializer.copy(left), right == null ? null : rightSerializer.copy(right), collector);
+		}
+
+		return result;
+	}
+
+	@SuppressWarnings("unchecked")
+	private <T> TypeComparator<T> buildComparatorFor(int input, ExecutionConfig executionConfig, TypeInformation<T> typeInformation) {
+		TypeComparator<T> comparator;
+		if (typeInformation instanceof AtomicType) {
+			comparator = ((AtomicType<T>) typeInformation).createComparator(true, executionConfig);
+		} else if (typeInformation instanceof CompositeType) {
+			int[] keyPositions = getKeyColumns(input);
+			boolean[] orders = new boolean[keyPositions.length];
+			Arrays.fill(orders, true);
+
+			comparator = ((CompositeType<T>) typeInformation).createComparator(keyPositions, orders, 0, executionConfig);
+		} else {
+			throw new RuntimeException("Type information for input of type " + typeInformation.getClass()
+					.getCanonicalName() + " is not supported. Could not generate a comparator.");
+		}
+		return comparator;
+	}
+
+	private static class OuterJoinListIterator<IN1, IN2> {
+
+
+		private static enum MatchStatus {
+			NONE_REMAINED, FIRST_REMAINED, SECOND_REMAINED, FIRST_EMPTY, SECOND_EMPTY
+		}
+
+		private OuterJoinType outerJoinType;
+
+		private ListKeyGroupedIterator<IN1> leftGroupedIterator;
+		private ListKeyGroupedIterator<IN2> rightGroupedIterator;
+		private Iterable<IN1> currLeftSubset;
+		private ResettableIterator currLeftIterator;
+		private Iterable<IN2> currRightSubset;
+		private ResettableIterator currRightIterator;
+
+		private MatchStatus matchStatus;
+		private GenericPairComparator<IN1, IN2> pairComparator;
+
+		private IN1 leftReturn;
+		private IN2 rightReturn;
+
+		public OuterJoinListIterator(List<IN1> leftInput, TypeSerializer<IN1> leftSerializer, final TypeComparator<IN1> leftComparator,
+				List<IN2> rightInput, TypeSerializer<IN2> rightSerializer, final TypeComparator<IN2> rightComparator,
+				OuterJoinType outerJoinType) {
+			this.outerJoinType = outerJoinType;
+			pairComparator = new GenericPairComparator<>(leftComparator, rightComparator);
+			leftGroupedIterator = new ListKeyGroupedIterator<>(leftInput, leftSerializer, leftComparator);
+			rightGroupedIterator = new ListKeyGroupedIterator<>(rightInput, rightSerializer, rightComparator);
+			// ----------------------------------------------------------------
+			// Sort
+			// ----------------------------------------------------------------
+			Collections.sort(leftInput, new Comparator<IN1>() {
+				@Override
+				public int compare(IN1 o1, IN1 o2) {
+					return leftComparator.compare(o1, o2);
+				}
+			});
+
+			Collections.sort(rightInput, new Comparator<IN2>() {
+				@Override
+				public int compare(IN2 o1, IN2 o2) {
+					return rightComparator.compare(o1, o2);
+				}
+			});
+
+		}
+
+		@SuppressWarnings("unchecked")
+		private boolean next() throws IOException {
+			boolean hasMoreElements;
+			if ((currLeftIterator == null || !currLeftIterator.hasNext()) && (currRightIterator == null || !currRightIterator.hasNext())) {
+				hasMoreElements = nextGroups(outerJoinType);
+				if (hasMoreElements) {
+					if (outerJoinType != OuterJoinType.LEFT) {
+						currLeftIterator = new ListIteratorWrapper(currLeftSubset.iterator());
+					}
+					leftReturn = (IN1) currLeftIterator.next();
+					if (outerJoinType != OuterJoinType.RIGHT) {
+						currRightIterator = new ListIteratorWrapper(currRightSubset.iterator());
+					}
+					rightReturn = (IN2) currRightIterator.next();
+					return true;
+				} else {
+					//no more elements
+					return false;
+				}
+			} else if (currLeftIterator.hasNext() && !currRightIterator.hasNext()) {
+				leftReturn = (IN1) currLeftIterator.next();
+				currRightIterator.reset();
+				rightReturn = (IN2) currRightIterator.next();
+				return true;
+			} else {
+				rightReturn = (IN2) currRightIterator.next();
+				return true;
+			}
+		}
+
+		private boolean nextGroups(OuterJoinType outerJoinType) throws IOException {
+			if (outerJoinType == OuterJoinType.FULL) {
+				return nextGroups();
+			} else if (outerJoinType == OuterJoinType.LEFT) {
+				boolean leftContainsElements = false;
+				while (!leftContainsElements && nextGroups()) {
+					currLeftIterator = new ListIteratorWrapper(currLeftSubset.iterator());
+					if (currLeftIterator.next() != null) {
+						leftContainsElements = true;
+					}
+					currLeftIterator.reset();
+				}
+				return leftContainsElements;
+			} else if (outerJoinType == OuterJoinType.RIGHT) {
+				boolean rightContainsElements = false;
+				while (!rightContainsElements && nextGroups()) {
+					currRightIterator = new ListIteratorWrapper(currRightSubset.iterator());
+					if (currRightIterator.next() != null) {
+						rightContainsElements = true;
+					}
+					currRightIterator.reset();
+				}
+				return rightContainsElements;
+			} else {
+				throw new IllegalArgumentException("Outer join of type '" + outerJoinType + "' not supported.");
+			}
+		}
+
+		private boolean nextGroups() throws IOException {
+			boolean firstEmpty = true;
+			boolean secondEmpty = true;
+
+			if (this.matchStatus != MatchStatus.FIRST_EMPTY) {
+				if (this.matchStatus == MatchStatus.FIRST_REMAINED) {
+					// comparator is still set correctly
+					firstEmpty = false;
+				} else {
+					if (this.leftGroupedIterator.nextKey()) {
+						this.pairComparator.setReference(leftGroupedIterator.getValues().getCurrent());
+						firstEmpty = false;
+					}
+				}
+			}
+
+			if (this.matchStatus != MatchStatus.SECOND_EMPTY) {
+				if (this.matchStatus == MatchStatus.SECOND_REMAINED) {
+					secondEmpty = false;
+				} else {
+					if (rightGroupedIterator.nextKey()) {
+						secondEmpty = false;
+					}
+				}
+			}
+
+			if (firstEmpty && secondEmpty) {
+				// both inputs are empty
+				return false;
+			} else if (firstEmpty && !secondEmpty) {
+				// input1 is empty, input2 not
+				this.currLeftSubset = Collections.singleton(null);
+				this.currRightSubset = this.rightGroupedIterator.getValues();
+				this.matchStatus = MatchStatus.FIRST_EMPTY;
+				return true;
+			} else if (!firstEmpty && secondEmpty) {
+				// input1 is not empty, input 2 is empty
+				this.currLeftSubset = this.leftGroupedIterator.getValues();
+				this.currRightSubset = Collections.singleton(null);
+				this.matchStatus = MatchStatus.SECOND_EMPTY;
+				return true;
+			} else {
+				// both inputs are not empty
+				final int comp = this.pairComparator.compareToReference(rightGroupedIterator.getValues().getCurrent());
+
+				if (0 == comp) {
+					// keys match
+					this.currLeftSubset = this.leftGroupedIterator.getValues();
+					this.currRightSubset = this.rightGroupedIterator.getValues();
+					this.matchStatus = MatchStatus.NONE_REMAINED;
+				} else if (0 < comp) {
+					// key1 goes first
+					this.currLeftSubset = this.leftGroupedIterator.getValues();
+					this.currRightSubset = Collections.singleton(null);
+					this.matchStatus = MatchStatus.SECOND_REMAINED;
+				} else {
+					// key 2 goes first
+					this.currLeftSubset = Collections.singleton(null);
+					this.currRightSubset = this.rightGroupedIterator.getValues();
+					this.matchStatus = MatchStatus.FIRST_REMAINED;
+				}
+				return true;
+			}
+		}
+
+		private IN1 getLeft() {
+			return leftReturn;
+		}
+
+		private IN2 getRight() {
+			return rightReturn;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
new file mode 100644
index 0000000..679e4ce
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.operators.base;
+
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+@SuppressWarnings("serial")
+public class OuterJoinOperatorBaseTest implements Serializable {
+
+	private final FlatJoinFunction<String, String, String> joiner = new FlatJoinFunction<String, String, String>() {
+		@Override
+		public void join(String first, String second, Collector<String> out) throws Exception {
+			out.collect(Joiner.on(',').join(String.valueOf(first), String.valueOf(second)));
+		}
+	};
+
+	@SuppressWarnings({"rawtypes", "unchecked"})
+	private final OuterJoinOperatorBase<String, String, String, FlatJoinFunction<String, String, String>> baseOperator =
+			new OuterJoinOperatorBase(joiner,
+					new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO,
+							BasicTypeInfo.STRING_TYPE_INFO), new int[0], new int[0], "TestJoiner", null);
+
+	@Test
+	public void testFullOuterJoinWithoutMatchingPartners() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
+		final List<String> rightInput = Arrays.asList("oof", "rab", "raboof");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("bar,null", "foo,null", "foobar,null", "null,oof", "null,rab", "null,raboof");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testFullOuterJoinWithFullMatchingKeys() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
+		final List<String> rightInput = Arrays.asList("bar", "foobar", "foo");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("bar,bar", "foo,foo", "foobar,foobar");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testFullOuterJoinWithEmptyLeftInput() throws Exception {
+		final List<String> leftInput = Arrays.asList();
+		final List<String> rightInput = Arrays.asList("foo", "bar", "foobar");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("null,bar", "null,foo", "null,foobar");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testFullOuterJoinWithEmptyRightInput() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
+		final List<String> rightInput = Arrays.asList();
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("bar,null", "foo,null", "foobar,null");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testFullOuterJoinWithPartialMatchingKeys() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
+		final List<String> rightInput = Arrays.asList("bar", "foo", "barfoo");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("bar,bar", "null,barfoo", "foo,foo", "foobar,null");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testFullOuterJoinBuildingCorrectCrossProducts() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "foo", "foo", "bar","bar", "foobar", "foobar");
+		final List<String> rightInput = Arrays.asList("foo", "foo", "bar", "bar", "bar", "barfoo", "barfoo");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
+		List<String> expected = Arrays.asList("bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar",
+				"null,barfoo", "null,barfoo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo",
+				"foobar,null", "foobar,null");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testLeftOuterJoin() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "foo", "foo", "bar","bar", "foobar", "foobar");
+		final List<String> rightInput = Arrays.asList("foo", "foo", "bar", "bar", "bar", "barfoo", "barfoo");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.LEFT);
+		List<String> expected = Arrays.asList("bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar",
+				"foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foobar,null", "foobar,null");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test
+	public void testRightOuterJoin() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "foo", "foo", "bar","bar", "foobar", "foobar");
+		final List<String> rightInput = Arrays.asList("foo", "foo", "bar", "bar", "bar", "barfoo", "barfoo");
+		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.RIGHT);
+		List<String> expected = Arrays.asList("bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar", "bar,bar",
+				"null,barfoo", "null,barfoo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo", "foo,foo");
+		testOuterJoin(leftInput, rightInput, expected);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testThatExceptionIsThrownForOuterJoinTypeNull() throws Exception {
+		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
+		final List<String> rightInput = Arrays.asList("bar", "foobar", "foo");
+
+		baseOperator.setOuterJoinType(null);
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.disableObjectReuse();
+		baseOperator.executeOnCollections(leftInput, rightInput, null, executionConfig);
+	}
+
+	private void testOuterJoin(List<String> leftInput, List<String> rightInput, List<String> expected) throws Exception {
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.disableObjectReuse();
+		List<String> resultSafe = baseOperator.executeOnCollections(leftInput, rightInput, null, executionConfig);
+		executionConfig.enableObjectReuse();
+		List<String> resultRegular = baseOperator.executeOnCollections(leftInput, rightInput, null, executionConfig);
+
+		assertEquals(expected, resultSafe);
+		assertEquals(expected, resultRegular);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
index 30786aa..d942b72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.runtime.operators;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
index 3cccab8..ae05d1e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.runtime.operators;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
index c93637e..6fc8abd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.runtime.operators;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
index d109cf8..74faeb3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.operators.sort;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -37,8 +38,6 @@ import java.util.Iterator;
  */
 public abstract class AbstractMergeOuterJoinIterator<T1, T2, O> extends AbstractMergeIterator<T1, T2, O> {
 
-	public enum OuterJoinType {LEFT, RIGHT, FULL}
-
 	private final OuterJoinType outerJoinType;
 
 	private boolean initialized = false;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
index db47f16..f2faa2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.operators.sort;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
index 8382b86..33d72d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.operators.sort;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
index 0c0e836..7b27fa9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.operators.sort;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.GenericPairComparator;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
@@ -37,7 +38,6 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.testutils.CollectionIterator;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
index 7272595..e930317 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
@@ -18,13 +18,13 @@
 
 package org.apache.flink.runtime.operators.sort;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b222276/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
index 2cec393..cca1b76 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
@@ -18,13 +18,13 @@
 
 package org.apache.flink.runtime.operators.sort;
 
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.Test;
 


[5/6] flink git commit: [FLINK-2576] [javaAPI] [scalaAPI] [optimizer] Add outerJoin to DataSet API (Java, Scala) and optimizer.

Posted by fh...@apache.org.
[FLINK-2576] [javaAPI] [scalaAPI] [optimizer] Add outerJoin to DataSet API (Java, Scala) and optimizer.

This closes #1138


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b00c1d7e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b00c1d7e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b00c1d7e

Branch: refs/heads/master
Commit: b00c1d7e7b83a3f2e42223fe97dd369755f884c3
Parents: 6b22227
Author: Johann Kovacs <me...@jkovacs.de>
Authored: Tue Sep 8 18:23:54 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Oct 9 16:19:21 2015 +0200

----------------------------------------------------------------------
 .../api/common/io/ReplicatingInputFormat.java   |   5 +-
 .../operators/base/InnerJoinOperatorBase.java   | 148 ++++
 .../common/operators/base/JoinOperatorBase.java | 155 +----
 .../operators/base/OuterJoinOperatorBase.java   |   2 +-
 .../base/InnerJoinOperatorBaseTest.java         | 141 ++++
 .../operators/base/JoinOperatorBaseTest.java    | 141 ----
 .../java/org/apache/flink/api/java/DataSet.java | 197 +++++-
 .../flink/api/java/operators/JoinOperator.java  | 693 +++++++------------
 .../operators/join/InnerJoinOperatorSets.java   | 154 +++++
 .../operators/join/JoinFunctionAssigner.java    |  39 ++
 .../java/operators/join/JoinOperatorSets.java   | 235 +++++++
 .../flink/api/java/operators/join/JoinType.java |  28 +
 .../PlanBothUnwrappingJoinOperator.java         |  66 --
 .../PlanLeftUnwrappingJoinOperator.java         |  64 --
 .../PlanRightUnwrappingJoinOperator.java        |  66 --
 .../translation/TupleLeftUnwrappingJoiner.java  |  40 ++
 .../translation/TupleRightUnwrappingJoiner.java |  40 ++
 .../translation/TupleUnwrappingJoiner.java      |  45 ++
 .../api/java/record/operators/JoinOperator.java |   4 +-
 .../base/InnerJoinOperatorBaseTest.java         | 120 ++++
 .../operators/base/JoinOperatorBaseTest.java    | 122 ----
 .../SemanticPropertiesProjectionTest.java       |   6 +-
 .../SemanticPropertiesTranslationTest.java      |  16 +-
 .../flink/api/java/operators/NamesTest.java     |   4 +-
 .../DeltaIterationTranslationTest.java          |   6 +-
 .../flink/optimizer/costs/CostEstimator.java    |   3 +
 .../apache/flink/optimizer/dag/JoinNode.java    |  18 +-
 .../apache/flink/optimizer/dag/MatchNode.java   | 167 -----
 .../flink/optimizer/dag/OuterJoinNode.java      | 130 ++++
 .../AbstractSortMergeJoinDescriptor.java        |  81 +++
 .../optimizer/operators/CoGroupDescriptor.java  |  25 +-
 .../operators/OperatorDescriptorDual.java       |  24 +
 .../SortMergeFullOuterJoinDescriptor.java       |  39 ++
 .../operators/SortMergeInnerJoinDescriptor.java |  44 ++
 .../operators/SortMergeJoinDescriptor.java      | 110 ---
 .../SortMergeLeftOuterJoinDescriptor.java       |  43 ++
 .../SortMergeRightOuterJoinDescriptor.java      |  43 ++
 .../traversals/GraphCreatingVisitor.java        |  15 +-
 .../optimizer/FeedbackPropertiesMatchTest.java  |   4 +-
 .../SemanticPropertiesAPIToPlanTest.java        |   4 +-
 .../JoinGlobalPropertiesCompatibilityTest.java  |   6 +-
 .../org/apache/flink/api/scala/DataSet.scala    | 117 +++-
 .../apache/flink/api/scala/joinDataSet.scala    | 139 +++-
 .../apache/flink/test/util/TestEnvironment.java |  21 +-
 .../test/javaApiOperators/OuterJoinITCase.java  | 605 ++++++++++++++++
 .../SemanticPropertiesTranslationTest.scala     |  10 +-
 .../api/scala/operators/OuterJoinITCase.scala   | 214 ++++++
 .../DeltaIterationTranslationTest.scala         |  10 +-
 48 files changed, 2946 insertions(+), 1463 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
index 3d0ea99..5c8eb27 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.common.io;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
@@ -33,7 +34,7 @@ import java.io.IOException;
  * This is done by assigning all {@link org.apache.flink.core.io.InputSplit}s generated by the
  * replicated InputFormat to each parallel instance.
  *
- * Replicated data can only be used as input for a {@link org.apache.flink.api.common.operators.base.JoinOperatorBase} or
+ * Replicated data can only be used as input for a {@link InnerJoinOperatorBase} or
  * {@link org.apache.flink.api.common.operators.base.CrossOperatorBase} with the same parallelism as the DataSource.
  * Before being used as an input to a Join or Cross operator, replicated data might be processed in local pipelines by
  * by Map-based operators with the same parallelism as the source. Map-based operators are
@@ -54,7 +55,7 @@ import java.io.IOException;
  *
  * @see org.apache.flink.api.common.io.InputFormat
  * @see org.apache.flink.api.common.io.RichInputFormat
- * @see org.apache.flink.api.common.operators.base.JoinOperatorBase
+ * @see InnerJoinOperatorBase
  * @see org.apache.flink.api.common.operators.base.CrossOperatorBase
  * @see org.apache.flink.api.common.operators.base.MapOperatorBase
  * @see org.apache.flink.api.common.operators.base.FlatMapOperatorBase

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java
new file mode 100644
index 0000000..8ec0c9e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java
@@ -0,0 +1,148 @@
+/*
+ * 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.operators.base;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.functions.util.CopyingListCollector;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
+import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.CompositeType;
+import org.apache.flink.api.common.typeutils.GenericPairComparator;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @see org.apache.flink.api.common.functions.FlatJoinFunction
+ */
+public class InnerJoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends JoinOperatorBase<IN1, IN2, OUT, FT> {
+
+	public InnerJoinOperatorBase(UserCodeWrapper<FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo,
+			int[] keyPositions1, int[] keyPositions2, String name) {
+		super(udf, operatorInfo, keyPositions1, keyPositions2, name);
+	}
+
+	public InnerJoinOperatorBase(FT udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo, int[] keyPositions1,
+			int[] keyPositions2, String name) {
+		super(new UserCodeObjectWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+	}
+
+	public InnerJoinOperatorBase(Class<? extends FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo,
+			int[] keyPositions1, int[] keyPositions2, String name) {
+		super(new UserCodeClassWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected List<OUT> executeOnCollections(List<IN1> inputData1, List<IN2> inputData2, RuntimeContext runtimeContext,
+			ExecutionConfig executionConfig) throws Exception {
+		FlatJoinFunction<IN1, IN2, OUT> function = userFunction.getUserCodeObject();
+
+		FunctionUtils.setFunctionRuntimeContext(function, runtimeContext);
+		FunctionUtils.openFunction(function, this.parameters);
+
+		TypeInformation<IN1> leftInformation = getOperatorInfo().getFirstInputType();
+		TypeInformation<IN2> rightInformation = getOperatorInfo().getSecondInputType();
+		TypeInformation<OUT> outInformation = getOperatorInfo().getOutputType();
+
+		TypeSerializer<IN1> leftSerializer = leftInformation.createSerializer(executionConfig);
+		TypeSerializer<IN2> rightSerializer = rightInformation.createSerializer(executionConfig);
+
+		TypeComparator<IN1> leftComparator;
+		TypeComparator<IN2> rightComparator;
+
+		if (leftInformation instanceof AtomicType) {
+			leftComparator = ((AtomicType<IN1>) leftInformation).createComparator(true, executionConfig);
+		} else if (leftInformation instanceof CompositeType) {
+			int[] keyPositions = getKeyColumns(0);
+			boolean[] orders = new boolean[keyPositions.length];
+			Arrays.fill(orders, true);
+
+			leftComparator = ((CompositeType<IN1>) leftInformation).createComparator(keyPositions, orders, 0, executionConfig);
+		} else {
+			throw new RuntimeException("Type information for left input of type " + leftInformation.getClass()
+					.getCanonicalName() + " is not supported. Could not generate a comparator.");
+		}
+
+		if (rightInformation instanceof AtomicType) {
+			rightComparator = ((AtomicType<IN2>) rightInformation).createComparator(true, executionConfig);
+		} else if (rightInformation instanceof CompositeType) {
+			int[] keyPositions = getKeyColumns(1);
+			boolean[] orders = new boolean[keyPositions.length];
+			Arrays.fill(orders, true);
+
+			rightComparator = ((CompositeType<IN2>) rightInformation).createComparator(keyPositions, orders, 0, executionConfig);
+		} else {
+			throw new RuntimeException("Type information for right input of type " + rightInformation.getClass()
+					.getCanonicalName() + " is not supported. Could not generate a comparator.");
+		}
+
+		TypePairComparator<IN1, IN2> pairComparator = new GenericPairComparator<IN1, IN2>(leftComparator, rightComparator);
+
+		List<OUT> result = new ArrayList<OUT>();
+		Collector<OUT> collector = new CopyingListCollector<OUT>(result, outInformation.createSerializer(executionConfig));
+
+		Map<Integer, List<IN2>> probeTable = new HashMap<Integer, List<IN2>>();
+
+		//Build hash table
+		for (IN2 element : inputData2) {
+			List<IN2> list = probeTable.get(rightComparator.hash(element));
+			if (list == null) {
+				list = new ArrayList<IN2>();
+				probeTable.put(rightComparator.hash(element), list);
+			}
+
+			list.add(element);
+		}
+
+		//Probing
+		for (IN1 left : inputData1) {
+			List<IN2> matchingHashes = probeTable.get(leftComparator.hash(left));
+
+			if (matchingHashes != null) {
+				pairComparator.setReference(left);
+				for (IN2 right : matchingHashes) {
+					if (pairComparator.equalToReference(right)) {
+						function.join(leftSerializer.copy(left), rightSerializer.copy(right), collector);
+					}
+				}
+			}
+		}
+
+		FunctionUtils.closeFunction(function);
+
+		return result;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java
----------------------------------------------------------------------
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 799496c..98194e1 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
@@ -18,207 +18,96 @@
 
 package org.apache.flink.api.common.operators.base;
 
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.CopyingListCollector;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
 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;
 import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
 import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.api.common.typeinfo.AtomicType;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.common.typeutils.GenericPairComparator;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.util.Collector;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * @see org.apache.flink.api.common.functions.FlatJoinFunction
- */
-public class JoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends DualInputOperator<IN1, IN2, OUT, FT> {
-	
+
+public abstract class JoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends DualInputOperator<IN1, IN2, OUT, FT> {
+
 	/**
 	 * An enumeration of hints, optionally usable to tell the system how exactly execute the join.
 	 */
 	public static enum JoinHint {
-		
+
 		/**
 		 * Leave the choice how to do the join to the optimizer. If in doubt, the
 		 * optimizer will choose a repartitioning join.
 		 */
 		OPTIMIZER_CHOOSES,
-		
+
 		/**
 		 * Hint that the first join input is much smaller than the second. This results in
 		 * broadcasting and hashing the first input, unless the optimizer infers that
 		 * prior existing partitioning is available that is even cheaper to exploit.
 		 */
 		BROADCAST_HASH_FIRST,
-		
+
 		/**
 		 * Hint that the second join input is much smaller than the first. This results in
 		 * broadcasting and hashing the second input, unless the optimizer infers that
 		 * prior existing partitioning is available that is even cheaper to exploit.
 		 */
 		BROADCAST_HASH_SECOND,
-		
+
 		/**
 		 * Hint that the first join input is a bit smaller than the second. This results in
 		 * repartitioning both inputs and hashing the first input, unless the optimizer infers that
 		 * prior existing partitioning and orders are available that are even cheaper to exploit.
 		 */
 		REPARTITION_HASH_FIRST,
-		
+
 		/**
-		 * Hint that the second join input is a bit smaller than the second. This results in
+		 * Hint that the second join input is a bit smaller than the first. This results in
 		 * repartitioning both inputs and hashing the second input, unless the optimizer infers that
 		 * prior existing partitioning and orders are available that are even cheaper to exploit.
 		 */
 		REPARTITION_HASH_SECOND,
-		
+
 		/**
 		 * Hint that the join should repartitioning both inputs and use sorting and merging
 		 * as the join strategy.
 		 */
 		REPARTITION_SORT_MERGE
-	};
-	
-	// --------------------------------------------------------------------------------------------
-	
-	
+	}
+
 	private JoinHint joinHint = JoinHint.OPTIMIZER_CHOOSES;
-	
 	private Partitioner<?> partitioner;
-	
-	
+
+
 	public JoinOperatorBase(UserCodeWrapper<FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo, int[] keyPositions1, int[] keyPositions2, String name) {
 		super(udf, operatorInfo, keyPositions1, keyPositions2, name);
 	}
 
 	public JoinOperatorBase(FT udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo, int[] keyPositions1, int[] keyPositions2, String name) {
-		super(new UserCodeObjectWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+		super(new UserCodeObjectWrapper<>(udf), operatorInfo, keyPositions1, keyPositions2, name);
 	}
-	
+
 	public JoinOperatorBase(Class<? extends FT> udf, BinaryOperatorInformation<IN1, IN2, OUT> operatorInfo, int[] keyPositions1, int[] keyPositions2, String name) {
-		super(new UserCodeClassWrapper<FT>(udf), operatorInfo, keyPositions1, keyPositions2, name);
+		super(new UserCodeClassWrapper<>(udf), operatorInfo, keyPositions1, keyPositions2, name);
 	}
-	
-	
+
+
 	public void setJoinHint(JoinHint joinHint) {
 		if (joinHint == null) {
 			throw new IllegalArgumentException("Join Hint must not be null.");
 		}
 		this.joinHint = joinHint;
 	}
-	
+
 	public JoinHint getJoinHint() {
 		return joinHint;
 	}
-	
+
 	public void setCustomPartitioner(Partitioner<?> partitioner) {
 		this.partitioner = partitioner;
 	}
-	
+
 	public Partitioner<?> getCustomPartitioner() {
 		return partitioner;
 	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("unchecked")
-	@Override
-	protected List<OUT> executeOnCollections(List<IN1> inputData1, List<IN2> inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) throws Exception {
-		FlatJoinFunction<IN1, IN2, OUT> function = userFunction.getUserCodeObject();
-
-		FunctionUtils.setFunctionRuntimeContext(function, runtimeContext);
-		FunctionUtils.openFunction(function, this.parameters);
-
-		TypeInformation<IN1> leftInformation = getOperatorInfo().getFirstInputType();
-		TypeInformation<IN2> rightInformation = getOperatorInfo().getSecondInputType();
-		TypeInformation<OUT> outInformation = getOperatorInfo().getOutputType();
 
-		TypeSerializer<IN1> leftSerializer = leftInformation.createSerializer(executionConfig);
-		TypeSerializer<IN2> rightSerializer = rightInformation.createSerializer(executionConfig);
-		
-		TypeComparator<IN1> leftComparator;
-		TypeComparator<IN2> rightComparator;
-
-		if (leftInformation instanceof AtomicType) {
-			leftComparator = ((AtomicType<IN1>) leftInformation).createComparator(true, executionConfig);
-		}
-		else if (leftInformation instanceof CompositeType) {
-			int[] keyPositions = getKeyColumns(0);
-			boolean[] orders = new boolean[keyPositions.length];
-			Arrays.fill(orders, true);
-
-			leftComparator = ((CompositeType<IN1>) leftInformation).createComparator(keyPositions, orders, 0, executionConfig);
-		}
-		else {
-			throw new RuntimeException("Type information for left input of type " + leftInformation.getClass()
-					.getCanonicalName() + " is not supported. Could not generate a comparator.");
-		}
-
-		if (rightInformation instanceof AtomicType) {
-			rightComparator = ((AtomicType<IN2>) rightInformation).createComparator(true, executionConfig);
-		}
-		else if (rightInformation instanceof CompositeType) {
-			int[] keyPositions = getKeyColumns(1);
-			boolean[] orders = new boolean[keyPositions.length];
-			Arrays.fill(orders, true);
-
-			rightComparator = ((CompositeType<IN2>) rightInformation).createComparator(keyPositions, orders, 0, executionConfig);
-		}
-		else {
-			throw new RuntimeException("Type information for right input of type " + rightInformation.getClass()
-					.getCanonicalName() + " is not supported. Could not generate a comparator.");
-		}
-
-		TypePairComparator<IN1, IN2> pairComparator = new GenericPairComparator<IN1, IN2>(leftComparator, rightComparator);
-
-		List<OUT> result = new ArrayList<OUT>();
-		Collector<OUT> collector = new CopyingListCollector<OUT>(result, outInformation.createSerializer(executionConfig));
-
-		Map<Integer, List<IN2>> probeTable = new HashMap<Integer, List<IN2>>();
-
-		//Build hash table
-		for (IN2 element: inputData2){
-			List<IN2> list = probeTable.get(rightComparator.hash(element));
-			if(list == null){
-				list = new ArrayList<IN2>();
-				probeTable.put(rightComparator.hash(element), list);
-			}
-
-			list.add(element);
-		}
-
-		//Probing
-		for (IN1 left: inputData1) {
-			List<IN2> matchingHashes = probeTable.get(leftComparator.hash(left));
-
-			if (matchingHashes != null) {
-				pairComparator.setReference(left);
-				for (IN2 right : matchingHashes) {
-					if (pairComparator.equalToReference(right)) {
-						function.join(leftSerializer.copy(left), rightSerializer.copy(right), collector);
-					}
-				}
-			}
-		}
-
-		FunctionUtils.closeFunction(function);
-
-		return result;
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
index 7666d10..02c8981 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java
@@ -45,7 +45,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
-public class OuterJoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends AbstractJoinOperatorBase<IN1, IN2, OUT, FT> {
+public class OuterJoinOperatorBase<IN1, IN2, OUT, FT extends FlatJoinFunction<IN1, IN2, OUT>> extends JoinOperatorBase<IN1, IN2, OUT, FT> {
 
 	public static enum OuterJoinType {LEFT, RIGHT, FULL}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
new file mode 100644
index 0000000..04505d0
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.operators.base;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+@SuppressWarnings("serial")
+public class InnerJoinOperatorBaseTest implements Serializable {
+
+	@Test
+	public void testJoinPlain(){
+		final FlatJoinFunction<String, String, Integer> joiner = new FlatJoinFunction<String, String, Integer>() {
+
+			@Override
+			public void join(String first, String second, Collector<Integer> out) throws Exception {
+				out.collect(first.length());
+				out.collect(second.length());
+			}
+		};
+
+		@SuppressWarnings({ "rawtypes", "unchecked" })
+		InnerJoinOperatorBase<String, String, Integer,
+						FlatJoinFunction<String, String,Integer> > base = new InnerJoinOperatorBase(joiner,
+				new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO,
+						BasicTypeInfo.INT_TYPE_INFO), new int[0], new int[0], "TestJoiner");
+
+		List<String> inputData1 = new ArrayList<String>(Arrays.asList("foo", "bar", "foobar"));
+		List<String> inputData2 = new ArrayList<String>(Arrays.asList("foobar", "foo"));
+		List<Integer> expected = new ArrayList<Integer>(Arrays.asList(3, 3, 6 ,6));
+
+		try {
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			executionConfig.disableObjectReuse();
+			List<Integer> resultSafe = base.executeOnCollections(inputData1, inputData2, null, executionConfig);
+			executionConfig.enableObjectReuse();
+			List<Integer> resultRegular = base.executeOnCollections(inputData1, inputData2, null, executionConfig);
+
+			assertEquals(expected, resultSafe);
+			assertEquals(expected, resultRegular);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testJoinRich(){
+		final AtomicBoolean opened = new AtomicBoolean(false);
+		final AtomicBoolean closed = new AtomicBoolean(false);
+		final String taskName = "Test rich join function";
+
+		final RichFlatJoinFunction<String, String, Integer> joiner = new RichFlatJoinFunction<String, String, Integer>() {
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				opened.compareAndSet(false, true);
+				assertEquals(0, getRuntimeContext().getIndexOfThisSubtask());
+				assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks());
+			}
+
+			@Override
+			public void close() throws Exception{
+				closed.compareAndSet(false, true);
+			}
+
+			@Override
+			public void join(String first, String second, Collector<Integer> out) throws Exception {
+				out.collect(first.length());
+				out.collect(second.length());
+			}
+		};
+
+		InnerJoinOperatorBase<String, String, Integer,
+						RichFlatJoinFunction<String, String, Integer>> base = new InnerJoinOperatorBase<String, String, Integer,
+										RichFlatJoinFunction<String, String, Integer>>(joiner, new BinaryOperatorInformation<String, String,
+				Integer>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO,
+				BasicTypeInfo.INT_TYPE_INFO), new int[0], new int[0], taskName);
+
+		final List<String> inputData1 = new ArrayList<String>(Arrays.asList("foo", "bar", "foobar"));
+		final List<String> inputData2 = new ArrayList<String>(Arrays.asList("foobar", "foo"));
+		final List<Integer> expected = new ArrayList<Integer>(Arrays.asList(3, 3, 6, 6));
+
+
+		try {
+			final HashMap<String, Accumulator<?, ?>> accumulatorMap = new HashMap<String, Accumulator<?, ?>>();
+			final HashMap<String, Future<Path>> cpTasks = new HashMap<>();
+
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			executionConfig.disableObjectReuse();
+			List<Integer> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig, cpTasks, accumulatorMap), executionConfig);
+			executionConfig.enableObjectReuse();
+			List<Integer> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig, cpTasks, accumulatorMap), executionConfig);
+
+			assertEquals(expected, resultSafe);
+			assertEquals(expected, resultRegular);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+
+		assertTrue(opened.get());
+		assertTrue(closed.get());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
deleted file mode 100644
index 6d4ff33..0000000
--- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
+++ /dev/null
@@ -1,141 +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.common.operators.base;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.RichFlatJoinFunction;
-import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
-import org.apache.flink.api.common.operators.BinaryOperatorInformation;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-@SuppressWarnings("serial")
-public class JoinOperatorBaseTest implements Serializable {
-
-	@Test
-	public void testJoinPlain(){
-		final FlatJoinFunction<String, String, Integer> joiner = new FlatJoinFunction<String, String, Integer>() {
-
-			@Override
-			public void join(String first, String second, Collector<Integer> out) throws Exception {
-				out.collect(first.length());
-				out.collect(second.length());
-			}
-		};
-
-		@SuppressWarnings({ "rawtypes", "unchecked" })
-		JoinOperatorBase<String, String, Integer,
-				FlatJoinFunction<String, String,Integer> > base = new JoinOperatorBase(joiner,
-				new BinaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO,
-						BasicTypeInfo.INT_TYPE_INFO), new int[0], new int[0], "TestJoiner");
-
-		List<String> inputData1 = new ArrayList<String>(Arrays.asList("foo", "bar", "foobar"));
-		List<String> inputData2 = new ArrayList<String>(Arrays.asList("foobar", "foo"));
-		List<Integer> expected = new ArrayList<Integer>(Arrays.asList(3, 3, 6 ,6));
-
-		try {
-			ExecutionConfig executionConfig = new ExecutionConfig();
-			executionConfig.disableObjectReuse();
-			List<Integer> resultSafe = base.executeOnCollections(inputData1, inputData2, null, executionConfig);
-			executionConfig.enableObjectReuse();
-			List<Integer> resultRegular = base.executeOnCollections(inputData1, inputData2, null, executionConfig);
-
-			assertEquals(expected, resultSafe);
-			assertEquals(expected, resultRegular);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testJoinRich(){
-		final AtomicBoolean opened = new AtomicBoolean(false);
-		final AtomicBoolean closed = new AtomicBoolean(false);
-		final String taskName = "Test rich join function";
-
-		final RichFlatJoinFunction<String, String, Integer> joiner = new RichFlatJoinFunction<String, String, Integer>() {
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				opened.compareAndSet(false, true);
-				assertEquals(0, getRuntimeContext().getIndexOfThisSubtask());
-				assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks());
-			}
-
-			@Override
-			public void close() throws Exception{
-				closed.compareAndSet(false, true);
-			}
-
-			@Override
-			public void join(String first, String second, Collector<Integer> out) throws Exception {
-				out.collect(first.length());
-				out.collect(second.length());
-			}
-		};
-
-		JoinOperatorBase<String, String, Integer,
-				RichFlatJoinFunction<String, String, Integer>> base = new JoinOperatorBase<String, String, Integer,
-				RichFlatJoinFunction<String, String, Integer>>(joiner, new BinaryOperatorInformation<String, String,
-				Integer>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO,
-				BasicTypeInfo.INT_TYPE_INFO), new int[0], new int[0], taskName);
-
-		final List<String> inputData1 = new ArrayList<String>(Arrays.asList("foo", "bar", "foobar"));
-		final List<String> inputData2 = new ArrayList<String>(Arrays.asList("foobar", "foo"));
-		final List<Integer> expected = new ArrayList<Integer>(Arrays.asList(3, 3, 6, 6));
-
-
-		try {
-			final HashMap<String, Accumulator<?, ?>> accumulatorMap = new HashMap<String, Accumulator<?, ?>>();
-			final HashMap<String, Future<Path>> cpTasks = new HashMap<>();
-
-			ExecutionConfig executionConfig = new ExecutionConfig();
-			executionConfig.disableObjectReuse();
-			List<Integer> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig, cpTasks, accumulatorMap), executionConfig);
-			executionConfig.enableObjectReuse();
-			List<Integer> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig, cpTasks, accumulatorMap), executionConfig);
-
-			assertEquals(expected, resultSafe);
-			assertEquals(expected, resultRegular);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-
-		assertTrue(opened.get());
-		assertTrue(closed.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
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 98a94c6..6c8df21 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
@@ -63,7 +63,7 @@ import org.apache.flink.api.java.operators.FlatMapOperator;
 import org.apache.flink.api.java.operators.GroupCombineOperator;
 import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets;
+import org.apache.flink.api.java.operators.join.InnerJoinOperatorSets;
 import org.apache.flink.api.java.operators.Keys;
 import org.apache.flink.api.java.operators.MapOperator;
 import org.apache.flink.api.java.operators.MapPartitionOperator;
@@ -75,6 +75,8 @@ import org.apache.flink.api.java.operators.SortPartitionOperator;
 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.join.JoinOperatorSets;
+import org.apache.flink.api.java.operators.join.JoinType;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
@@ -290,8 +292,8 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a Project transformation on a {@link Tuple} {@link DataSet}.<br/>
-	 * <b>Note: Only Tuple DataSets can be projected using field indexes.</b></br>
-	 * The transformation projects each Tuple of the DataSet onto a (sub)set of fields.</br>
+	 * <b>Note: Only Tuple DataSets can be projected using field indexes.</b><br/>
+	 * The transformation projects each Tuple of the DataSet onto a (sub)set of fields.<br/>
 	 * Additional fields can be added to the projection by calling {@link ProjectOperator#project(int[])}.
 	 *
 	 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
@@ -638,7 +640,7 @@ public abstract class DataSet<T> {
 	/**
 	 * Groups a {@link DataSet} using a {@link KeySelector} function. 
 	 * The KeySelector function is called for each element of the DataSet and extracts a single 
-	 *   key value on which the DataSet is grouped. </br>
+	 *   key value on which the DataSet is grouped. <br/>
 	 * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation 
 	 *   can be applied. 
 	 * <ul>
@@ -665,7 +667,7 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Groups a {@link Tuple} {@link DataSet} using field position keys.<br/> 
-	 * <b>Note: Field position keys only be specified for Tuple DataSets.</b></br>
+	 * <b>Note: Field position keys only be specified for Tuple DataSets.</b><br/>
 	 * The field position keys specify the fields of Tuples on which the DataSet is grouped.
 	 * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation 
 	 *   can be applied. 
@@ -725,83 +727,204 @@ public abstract class DataSet<T> {
 	 * Initiates a Join transformation. <br/>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.</br>
+	 *   joining elements into one DataSet.<br/>
 	 * 
-	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
+	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
-	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see JoinOperatorSets
+	 * @see InnerJoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> join(DataSet<R> other) {
-		return new JoinOperatorSets<T, R>(this, other);
+	public <R> InnerJoinOperatorSets<T, R> join(DataSet<R> other) {
+		return new InnerJoinOperatorSets<T, R>(this, other);
 	}
 	
 	/**
 	 * Initiates a Join transformation. <br/>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.</br>
+	 *   joining elements into one DataSet.<br/>
 	 * 
-	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
+	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
-	 * @param strategy The strategy that should be used execute the join. If {@code null} is give, then the
+	 * @param strategy The strategy that should be used execute the join. If {@code null} is given, then the
 	 *                 optimizer will pick the join strategy.
-	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see JoinOperatorSets
+	 * @see InnerJoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> join(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSets<T, R>(this, other, strategy);
+	public <R> InnerJoinOperatorSets<T, R> join(DataSet<R> other, JoinHint strategy) {
+		return new InnerJoinOperatorSets<T, R>(this, other, strategy);
 	}
 
 	/**
 	 * Initiates a Join transformation. <br/>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.</br>
+	 *   joining elements into one DataSet.<br/>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
-	 *   smaller than the first one.</br>
-	 * This method returns a {@link JoinOperatorSets} on which 
-	 *   {@link JoinOperatorSets#where(String...)} needs to be called to define the join key of the first 
+	 *   smaller than the first one.<br/>
+	 * This method returns a {@link InnerJoinOperatorSets} on which
+	 *   {@link InnerJoinOperatorSets#where(String...)} needs to be called to define the join key of the first
 	 *   joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
-	 * @return A JoinOperatorSets to continue the definition of the Join transformation.
+	 * @return A InnerJoinOperatorSets to continue the definition of the Join transformation.
 	 * 
-	 * @see JoinOperatorSets
+	 * @see InnerJoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> joinWithTiny(DataSet<R> other) {
-		return new JoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_SECOND);
+	public <R> InnerJoinOperatorSets<T, R> joinWithTiny(DataSet<R> other) {
+		return new InnerJoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_SECOND);
 	}
 	
 	/**
 	 * Initiates a Join transformation.<br/>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.</br>
+	 *   joining elements into one DataSet.<br/>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
-	 *   larger than the first one.</br>
-	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
+	 *   larger than the first one.<br/>
+	 * This method returns a {@link InnerJoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
 	 * @param other The other DataSet with which this DataSet is joined.
 	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
 	 * 
+	 * @see InnerJoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> InnerJoinOperatorSets<T, R> joinWithHuge(DataSet<R> other) {
+		return new InnerJoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+	/**
+	 * Initiates a Left Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of the <b>left</b> DataSet (i.e. {@code this}) that do not have a matching
+	 *   element on the other side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
 	 * @see JoinOperatorSets
 	 * @see DataSet
 	 */
-	public <R> JoinOperatorSets<T, R> joinWithHuge(DataSet<R> other) {
-		return new JoinOperatorSets<T, R>(this, other, JoinHint.BROADCAST_HASH_FIRST);
+	public <R> JoinOperatorSets<T, R> leftOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.LEFT_OUTER);
 	}
-	
+
+	/**
+	 * Initiates a Left Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of the <b>left</b> DataSet (i.e. {@code this}) that do not have a matching
+	 *   element on the other side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @param strategy The strategy that should be used execute the join. If {@code null} is given, then the
+	 *                 optimizer will pick the join strategy.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
+	 * @see JoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> JoinOperatorSets<T, R> leftOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSets<>(this, other, strategy, JoinType.LEFT_OUTER);
+	}
+
+	/**
+	 * Initiates a Right Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of the <b>right</b> DataSet (i.e. {@code other}) that do not have a matching
+	 *   element on {@code this} side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
+	 * @see JoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> JoinOperatorSets<T, R> rightOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.RIGHT_OUTER);
+	}
+
+	/**
+	 * Initiates a Right Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of the <b>right</b> DataSet (i.e. {@code other}) that do not have a matching
+	 *   element on {@code this} side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @param strategy The strategy that should be used execute the join. If {@code null} is given, then the
+	 *                 optimizer will pick the join strategy.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
+	 * @see JoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> JoinOperatorSets<T, R> rightOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSets<>(this, other, strategy, JoinType.RIGHT_OUTER);
+	}
+
+	/**
+	 * Initiates a Full Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of <b>both</b> DataSets that do not have a matching
+	 *   element on the opposing side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
+	 * @see JoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> JoinOperatorSets<T, R> fullOuterJoin(DataSet<R> other) {
+		return new JoinOperatorSets<>(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.FULL_OUTER);
+	}
+
+	/**
+	 * Initiates a Full Outer Join transformation.<br/>
+	 * An Outer Join transformation joins two elements of two
+	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
+	 *   joining elements into one DataSet.<br/>
+	 * Elements of <b>both</b> DataSets that do not have a matching
+	 *   element on the opposing side are joined with {@code null} and emitted to the
+	 *   resulting DataSet.
+	 *
+	 * @param other The other DataSet with which this DataSet is joined.
+	 * @param strategy The strategy that should be used execute the join. If {@code null} is given, then the
+	 *                 optimizer will pick the join strategy.
+	 * @return A JoinOperatorSet to continue the definition of the Join transformation.
+	 *
+	 * @see JoinOperatorSets
+	 * @see DataSet
+	 */
+	public <R> JoinOperatorSets<T, R> fullOuterJoin(DataSet<R> other, JoinHint strategy) {
+		return new JoinOperatorSets<>(this, other, strategy, JoinType.FULL_OUTER);
+	}
+
+
 	// --------------------------------------------------------------------------------------------
 	//  Co-Grouping
 	// --------------------------------------------------------------------------------------------
@@ -812,9 +935,9 @@ public abstract class DataSet<T> {
 	 *   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 org.apache.flink.api.common.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.</br>
+	 *   is called with an empty group for the non-existing group.<br/>
 	 * The CoGroupFunction can iterate over the elements of both groups and return any number 
-	 *   of elements including none.</br>
+	 *   of elements including none.<br/>
 	 * This method returns a {@link CoGroupOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 * 
@@ -860,7 +983,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.</br>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.
@@ -890,7 +1013,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.</br>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.
@@ -920,7 +1043,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.</br>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.


[3/6] flink git commit: [FLINK-2576] [javaAPI] [scalaAPI] [optimizer] Add outerJoin to DataSet API (Java, Scala) and optimizer.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
deleted file mode 100644
index 7390af2..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java
+++ /dev/null
@@ -1,122 +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.common.operators.base;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
-import org.apache.flink.api.common.operators.BinaryOperatorInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.Future;
-
-@SuppressWarnings({ "unchecked", "serial" })
-public class JoinOperatorBaseTest implements Serializable {
-
-	
-	@Test
-	public void testTupleBaseJoiner(){
-		final FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double, String>> joiner =
-					new FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double, String>>()
-		{
-			@Override
-			public void join(Tuple3<String, Double, Integer> first, Tuple2<Integer, String> second, Collector<Tuple2<Double, String>> out) {
-				Tuple3<String, Double, Integer> fst = (Tuple3<String, Double, Integer>)first;
-				Tuple2<Integer, String> snd = (Tuple2<Integer, String>)second;
-
-				assertEquals(fst.f0, snd.f1);
-				assertEquals(fst.f2, snd.f0);
-
-				out.collect(new Tuple2<Double, String>(fst.f1, snd.f0.toString()));
-			}
-		};
-
-		final TupleTypeInfo<Tuple3<String, Double, Integer>> leftTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo
-				(String.class, Double.class, Integer.class);
-		final TupleTypeInfo<Tuple2<Integer, String>> rightTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo(Integer.class,
-				String.class);
-		final TupleTypeInfo<Tuple2<Double, String>> outTypeInfo = TupleTypeInfo.getBasicTupleTypeInfo(Double.class,
-				String.class);
-
-		final int[] leftKeys = new int[]{0,2};
-		final int[] rightKeys = new int[]{1,0};
-
-		final String taskName = "Collection based tuple joiner";
-
-		final BinaryOperatorInformation<Tuple3<String, Double, Integer>, Tuple2<Integer, String>, Tuple2<Double,
-				String>> binaryOpInfo = new BinaryOperatorInformation<Tuple3<String, Double, Integer>, Tuple2<Integer,
-				String>, Tuple2<Double, String>>(leftTypeInfo, rightTypeInfo, outTypeInfo);
-
-		final JoinOperatorBase<Tuple3<String, Double, Integer>, Tuple2<Integer,
-				String>, Tuple2<Double, String>, FlatJoinFunction<Tuple3<String, Double, Integer>, Tuple2<Integer,
-				String>, Tuple2<Double, String>>> base = new JoinOperatorBase<Tuple3<String, Double, Integer>,
-				Tuple2<Integer, String>, Tuple2<Double, String>, FlatJoinFunction<Tuple3<String, Double, Integer>,
-				Tuple2<Integer, String>, Tuple2<Double, String>>>(joiner, binaryOpInfo, leftKeys, rightKeys, taskName);
-
-		final List<Tuple3<String, Double, Integer> > inputData1 = new ArrayList<Tuple3<String, Double,
-				Integer>>(Arrays.asList(
-				new Tuple3<String, Double, Integer>("foo", 42.0, 1),
-				new Tuple3<String,Double, Integer>("bar", 1.0, 2),
-				new Tuple3<String, Double, Integer>("bar", 2.0, 3),
-				new Tuple3<String, Double, Integer>("foobar", 3.0, 4),
-				new Tuple3<String, Double, Integer>("bar", 3.0, 3)
-		));
-
-		final List<Tuple2<Integer, String>> inputData2 = new ArrayList<Tuple2<Integer, String>>(Arrays.asList(
-				new Tuple2<Integer, String>(3, "bar"),
-				new Tuple2<Integer, String>(4, "foobar"),
-				new Tuple2<Integer, String>(2, "foo")
-		));
-		final Set<Tuple2<Double, String>> expected = new HashSet<Tuple2<Double, String>>(Arrays.asList(
-				new Tuple2<Double, String>(2.0, "3"),
-				new Tuple2<Double, String>(3.0, "3"),
-				new Tuple2<Double, String>(3.0, "4")
-		));
-
-		try {
-			ExecutionConfig executionConfig = new ExecutionConfig();
-			executionConfig.disableObjectReuse();
-			List<Tuple2<Double, String>> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig, new HashMap<String, Future<Path>>(), new HashMap<String, Accumulator<?, ?>>()), executionConfig);
-			executionConfig.enableObjectReuse();
-			List<Tuple2<Double, String>> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig, new HashMap<String, Future<Path>>(), new HashMap<String, Accumulator<?, ?>>()), executionConfig);
-
-			assertEquals(expected, new HashSet<Tuple2<Double, String>>(resultSafe));
-			assertEquals(expected, new HashSet<Tuple2<Double, String>>(resultRegular));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java
index e890b4e..916086b 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.GenericDataSinkBase;
 import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
 import org.apache.flink.api.common.operators.base.CrossOperatorBase;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.translation.PlanProjectOperator;
@@ -138,7 +138,7 @@ public class SemanticPropertiesProjectionTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> projectJoinOperator = ((JoinOperatorBase<?, ?, ?, ?>) sink.getInput());
+		InnerJoinOperatorBase<?, ?, ?, ?> projectJoinOperator = ((InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput());
 
 		DualInputSemanticProperties props = projectJoinOperator.getSemanticProperties();
 
@@ -166,7 +166,7 @@ public class SemanticPropertiesProjectionTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> projectJoinOperator = ((JoinOperatorBase<?, ?, ?, ?>) sink.getInput());
+		InnerJoinOperatorBase<?, ?, ?, ?> projectJoinOperator = ((InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput());
 
 		DualInputSemanticProperties props = projectJoinOperator.getSemanticProperties();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java
----------------------------------------------------------------------
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 33b3958..d01ca32 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,7 +28,7 @@ import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.GenericDataSinkBase;
 import org.apache.flink.api.common.operators.SemanticProperties;
 import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 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.FunctionAnnotation.ForwardedFields;
@@ -265,7 +265,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(0, 0));
@@ -292,7 +292,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(1, 0));
@@ -319,7 +319,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(1, 0));
@@ -352,7 +352,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(0, 1));
@@ -382,7 +382,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(0, 1));
@@ -410,7 +410,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getForwardingTargetFields(0, 0));
@@ -440,7 +440,7 @@ public class SemanticPropertiesTranslationTest {
 		Plan plan = env.createProgramPlan();
 
 		GenericDataSinkBase<?> sink = plan.getDataSinks().iterator().next();
-		JoinOperatorBase<?, ?, ?, ?> join = (JoinOperatorBase<?, ?, ?, ?>) sink.getInput();
+		InnerJoinOperatorBase<?, ?, ?, ?> join = (InnerJoinOperatorBase<?, ?, ?, ?>) sink.getInput();
 
 		DualInputSemanticProperties semantics = join.getSemanticProperties();
 		assertNotNull(semantics.getReadFields(0));

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java
index b8663ce..c14f175 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
@@ -102,7 +102,7 @@ public class NamesTest implements Serializable {
 		plan.accept(new Visitor<Operator<?>>() {
 			@Override
 			public boolean preVisit(Operator<?> visitable) {
-				if(visitable instanceof JoinOperatorBase) {
+				if(visitable instanceof InnerJoinOperatorBase) {
 					Assert.assertEquals("Join at testJoinWith(NamesTest.java:93)", visitable.getName());
 				}
 				return true;

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
----------------------------------------------------------------------
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 f9ce82f..fd60bc6 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
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
 import org.apache.flink.api.common.operators.GenericDataSinkBase;
 import org.apache.flink.api.common.operators.base.DeltaIterationBase;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
@@ -123,8 +123,8 @@ public class DeltaIterationTranslationTest implements java.io.Serializable {
 			assertEquals(ITERATION_NAME, iteration.getName());
 			
 			MapOperatorBase<?, ?, ?> nextWorksetMapper = (MapOperatorBase<?, ?, ?>) iteration.getNextWorkset();
-			JoinOperatorBase<?, ?, ?, ?> solutionSetJoin = (JoinOperatorBase<?, ?, ?, ?>) iteration.getSolutionSetDelta();
-			JoinOperatorBase<?, ?, ?, ?> worksetSelfJoin = (JoinOperatorBase<?, ?, ?, ?>) solutionSetJoin.getFirstInput();
+			InnerJoinOperatorBase<?, ?, ?, ?> solutionSetJoin = (InnerJoinOperatorBase<?, ?, ?, ?>) iteration.getSolutionSetDelta();
+			InnerJoinOperatorBase<?, ?, ?, ?> worksetSelfJoin = (InnerJoinOperatorBase<?, ?, ?, ?>) solutionSetJoin.getFirstInput();
 			MapOperatorBase<?, ?, ?> worksetMapper = (MapOperatorBase<?, ?, ?>) worksetSelfJoin.getFirstInput();
 			
 			assertEquals(IdentityMapper.class, worksetMapper.getUserCodeWrapper().getUserCodeClass());

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java
index 6a3ff09..553c127 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java
@@ -196,6 +196,9 @@ public abstract class CostEstimator {
 			
 			break;
 		case INNER_MERGE:
+		case FULL_OUTER_MERGE:
+		case LEFT_OUTER_MERGE:
+		case RIGHT_OUTER_MERGE:
 			addLocalMergeCost(firstInput, secondInput, driverCosts, costWeight);
 			break;
 		case HYBRIDHASH_BUILD_FIRST:

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java
index 02c9b5b..383bbe1 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java
@@ -23,7 +23,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
 import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.optimizer.DataStatistics;
@@ -32,7 +32,7 @@ import org.apache.flink.optimizer.operators.AbstractJoinDescriptor;
 import org.apache.flink.optimizer.operators.HashJoinBuildFirstProperties;
 import org.apache.flink.optimizer.operators.HashJoinBuildSecondProperties;
 import org.apache.flink.optimizer.operators.OperatorDescriptorDual;
-import org.apache.flink.optimizer.operators.SortMergeJoinDescriptor;
+import org.apache.flink.optimizer.operators.SortMergeInnerJoinDescriptor;
 import org.apache.flink.configuration.Configuration;
 
 /**
@@ -47,7 +47,7 @@ public class JoinNode extends TwoInputNode {
 	 * 
 	 * @param joinOperatorBase The join operator object.
 	 */
-	public JoinNode(JoinOperatorBase<?, ?, ?, ?> joinOperatorBase) {
+	public JoinNode(InnerJoinOperatorBase<?, ?, ?, ?> joinOperatorBase) {
 		super(joinOperatorBase);
 		
 		this.dataProperties = getDataProperties(joinOperatorBase,
@@ -62,8 +62,8 @@ public class JoinNode extends TwoInputNode {
 	 * @return The contract.
 	 */
 	@Override
-	public JoinOperatorBase<?, ?, ?, ?> getOperator() {
-		return (JoinOperatorBase<?, ?, ?, ?>) super.getOperator();
+	public InnerJoinOperatorBase<?, ?, ?, ?> getOperator() {
+		return (InnerJoinOperatorBase<?, ?, ?, ?>) super.getOperator();
 	}
 
 	@Override
@@ -111,7 +111,7 @@ public class JoinNode extends TwoInputNode {
 		}
 	}
 	
-	private List<OperatorDescriptorDual> getDataProperties(JoinOperatorBase<?, ?, ?, ?> joinOperatorBase, JoinHint joinHint,
+	private List<OperatorDescriptorDual> getDataProperties(InnerJoinOperatorBase<?, ?, ?, ?> joinOperatorBase, JoinHint joinHint,
 			Partitioner<?> customPartitioner)
 	{
 		// see if an internal hint dictates the strategy to use
@@ -125,7 +125,7 @@ public class JoinNode extends TwoInputNode {
 				Optimizer.HINT_LOCAL_STRATEGY_SORT_SECOND_MERGE.equals(localStrategy) ||
 				Optimizer.HINT_LOCAL_STRATEGY_MERGE.equals(localStrategy) )
 			{
-				fixedDriverStrat = new SortMergeJoinDescriptor(this.keys1, this.keys2);
+				fixedDriverStrat = new SortMergeInnerJoinDescriptor(this.keys1, this.keys2);
 			}
 			else if (Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST.equals(localStrategy)) {
 				fixedDriverStrat = new HashJoinBuildFirstProperties(this.keys1, this.keys2);
@@ -164,10 +164,10 @@ public class JoinNode extends TwoInputNode {
 					list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2, false, false, true));
 					break;
 				case REPARTITION_SORT_MERGE:
-					list.add(new SortMergeJoinDescriptor(this.keys1, this.keys2, false, false, true));
+					list.add(new SortMergeInnerJoinDescriptor(this.keys1, this.keys2, false, false, true));
 					break;
 				case OPTIMIZER_CHOOSES:
-					list.add(new SortMergeJoinDescriptor(this.keys1, this.keys2));
+					list.add(new SortMergeInnerJoinDescriptor(this.keys1, this.keys2));
 					list.add(new HashJoinBuildFirstProperties(this.keys1, this.keys2));
 					list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2));
 					break;

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MatchNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MatchNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MatchNode.java
deleted file mode 100644
index ee8ab05..0000000
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MatchNode.java
+++ /dev/null
@@ -1,167 +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.optimizer.dag;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
-import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.operators.HashJoinBuildFirstProperties;
-import org.apache.flink.optimizer.operators.HashJoinBuildSecondProperties;
-import org.apache.flink.optimizer.operators.OperatorDescriptorDual;
-import org.apache.flink.optimizer.operators.SortMergeJoinDescriptor;
-import org.apache.flink.configuration.Configuration;
-
-/**
- * The Optimizer representation of a join operator.
- */
-public class MatchNode extends TwoInputNode {
-	
-	private List<OperatorDescriptorDual> dataProperties;
-	
-	/**
-	 * Creates a new MatchNode for the given join operator.
-	 * 
-	 * @param joinOperatorBase The join operator object.
-	 */
-	public MatchNode(JoinOperatorBase<?, ?, ?, ?> joinOperatorBase) {
-		super(joinOperatorBase);
-		this.dataProperties = getDataProperties(joinOperatorBase, joinOperatorBase.getJoinHint());
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the contract object for this match node.
-	 * 
-	 * @return The contract.
-	 */
-	@Override
-	public JoinOperatorBase<?, ?, ?, ?> getOperator() {
-		return (JoinOperatorBase<?, ?, ?, ?>) super.getOperator();
-	}
-
-	@Override
-	public String getOperatorName() {
-		return "Join";
-	}
-
-	@Override
-	protected List<OperatorDescriptorDual> getPossibleProperties() {
-		return this.dataProperties;
-	}
-	
-	public void makeJoinWithSolutionSet(int solutionsetInputIndex) {
-		OperatorDescriptorDual op;
-		if (solutionsetInputIndex == 0) {
-			op = new HashJoinBuildFirstProperties(this.keys1, this.keys2);
-		} else if (solutionsetInputIndex == 1) {
-			op = new HashJoinBuildSecondProperties(this.keys1, this.keys2);
-		} else {
-			throw new IllegalArgumentException();
-		}
-		
-		this.dataProperties = Collections.singletonList(op);
-	}
-	
-	/**
-	 * The default estimates build on the principle of inclusion: The smaller input key domain is included in the larger
-	 * input key domain. We also assume that every key from the larger input has one join partner in the smaller input.
-	 * The result cardinality is hence the larger one.
-	 */
-	@Override
-	protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) {
-		long card1 = getFirstPredecessorNode().getEstimatedNumRecords();
-		long card2 = getSecondPredecessorNode().getEstimatedNumRecords();
-		this.estimatedNumRecords = (card1 < 0 || card2 < 0) ? -1 : Math.max(card1, card2);
-		
-		if (this.estimatedNumRecords >= 0) {
-			float width1 = getFirstPredecessorNode().getEstimatedAvgWidthPerOutputRecord();
-			float width2 = getSecondPredecessorNode().getEstimatedAvgWidthPerOutputRecord();
-			float width = (width1 <= 0 || width2 <= 0) ? -1 : width1 + width2;
-			
-			if (width > 0) {
-				this.estimatedOutputSize = (long) (width * this.estimatedNumRecords);
-			}
-		}
-	}
-	
-	private List<OperatorDescriptorDual> getDataProperties(JoinOperatorBase<?, ?, ?, ?> joinOperatorBase, JoinHint joinHint) {
-		// see if an internal hint dictates the strategy to use
-		Configuration conf = joinOperatorBase.getParameters();
-		String localStrategy = conf.getString(Optimizer.HINT_LOCAL_STRATEGY, null);
-
-		if (localStrategy != null) {
-			final OperatorDescriptorDual fixedDriverStrat;
-			if (Optimizer.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE.equals(localStrategy) ||
-				Optimizer.HINT_LOCAL_STRATEGY_SORT_FIRST_MERGE.equals(localStrategy) ||
-				Optimizer.HINT_LOCAL_STRATEGY_SORT_SECOND_MERGE.equals(localStrategy) ||
-				Optimizer.HINT_LOCAL_STRATEGY_MERGE.equals(localStrategy) )
-			{
-				fixedDriverStrat = new SortMergeJoinDescriptor(this.keys1, this.keys2);
-			} else if (Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST.equals(localStrategy)) {
-				fixedDriverStrat = new HashJoinBuildFirstProperties(this.keys1, this.keys2);
-			} else if (Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND.equals(localStrategy)) {
-				fixedDriverStrat = new HashJoinBuildSecondProperties(this.keys1, this.keys2);
-			} else {
-				throw new CompilerException("Invalid local strategy hint for match contract: " + localStrategy);
-			}
-			ArrayList<OperatorDescriptorDual> list = new ArrayList<OperatorDescriptorDual>();
-			list.add(fixedDriverStrat);
-			return list;
-		}
-		else {
-			ArrayList<OperatorDescriptorDual> list = new ArrayList<OperatorDescriptorDual>();
-			
-			joinHint = joinHint == null ? JoinHint.OPTIMIZER_CHOOSES : joinHint;
-			
-			switch (joinHint) {
-				case BROADCAST_HASH_FIRST:
-					list.add(new HashJoinBuildFirstProperties(this.keys1, this.keys2, true, false, false));
-					break;
-				case BROADCAST_HASH_SECOND:
-					list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2, false, true, false));
-					break;
-				case REPARTITION_HASH_FIRST:
-					list.add(new HashJoinBuildFirstProperties(this.keys1, this.keys2, false, false, true));
-					break;
-				case REPARTITION_HASH_SECOND:
-					list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2, false, false, true));
-					break;
-				case REPARTITION_SORT_MERGE:
-					list.add(new SortMergeJoinDescriptor(this.keys1, this.keys2, false, false, true));
-					break;
-				case OPTIMIZER_CHOOSES:
-					list.add(new SortMergeJoinDescriptor(this.keys1, this.keys2));
-					list.add(new HashJoinBuildFirstProperties(this.keys1, this.keys2));
-					list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2));
-					break;
-				default:
-					throw new CompilerException("Unrecognized join hint: " + joinHint);
-			}
-			
-			return list;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java
new file mode 100644
index 0000000..ebdfcc8
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java
@@ -0,0 +1,130 @@
+/*
+ * 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.optimizer.dag;
+
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType;
+import org.apache.flink.optimizer.CompilerException;
+import org.apache.flink.optimizer.DataStatistics;
+import org.apache.flink.optimizer.operators.AbstractJoinDescriptor;
+import org.apache.flink.optimizer.operators.OperatorDescriptorDual;
+import org.apache.flink.optimizer.operators.SortMergeFullOuterJoinDescriptor;
+import org.apache.flink.optimizer.operators.SortMergeLeftOuterJoinDescriptor;
+import org.apache.flink.optimizer.operators.SortMergeRightOuterJoinDescriptor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class OuterJoinNode extends TwoInputNode {
+
+	private List<OperatorDescriptorDual> dataProperties;
+
+	/**
+	 * Creates a new two input node for the optimizer plan, representing the given operator.
+	 *
+	 * @param operator The operator that the optimizer DAG node should represent.
+	 */
+	public OuterJoinNode(OuterJoinOperatorBase<?, ?, ?, ?> operator) {
+		super(operator);
+
+		this.dataProperties = getDataProperties();
+	}
+
+	private List<OperatorDescriptorDual> getDataProperties() {
+		OuterJoinOperatorBase<?, ?, ?, ?> operator = getOperator();
+
+		OuterJoinType type = operator.getOuterJoinType();
+
+		JoinHint joinHint = operator.getJoinHint();
+		joinHint = joinHint == null ? JoinHint.OPTIMIZER_CHOOSES : joinHint;
+
+		List<OperatorDescriptorDual> list = new ArrayList<>();
+		switch (joinHint) {
+			case OPTIMIZER_CHOOSES:
+				list.add(getSortMergeDescriptor(type, true));
+				break;
+			case REPARTITION_SORT_MERGE:
+				list.add(getSortMergeDescriptor(type, false));
+				break;
+			case REPARTITION_HASH_FIRST:
+			case REPARTITION_HASH_SECOND:
+			case BROADCAST_HASH_FIRST:
+			case BROADCAST_HASH_SECOND:
+			default:
+				throw new CompilerException("Invalid join hint: " + joinHint + " for outer join type: " + type);
+		}
+
+		Partitioner<?> customPartitioner = operator.getCustomPartitioner();
+		if (customPartitioner != null) {
+			for (OperatorDescriptorDual desc : list) {
+				((AbstractJoinDescriptor) desc).setCustomPartitioner(customPartitioner);
+			}
+		}
+		return list;
+	}
+
+	private OperatorDescriptorDual getSortMergeDescriptor(OuterJoinType type, boolean broadcastAllowed) {
+		if (type == OuterJoinType.FULL) {
+			return new SortMergeFullOuterJoinDescriptor(this.keys1, this.keys2);
+		} else if (type == OuterJoinType.LEFT) {
+			return new SortMergeLeftOuterJoinDescriptor(this.keys1, this.keys2, broadcastAllowed);
+		} else {
+			return new SortMergeRightOuterJoinDescriptor(this.keys1, this.keys2, broadcastAllowed);
+		}
+	}
+
+	@Override
+	public OuterJoinOperatorBase<?, ?, ?, ?> getOperator() {
+		return (OuterJoinOperatorBase<?, ?, ?, ?>) super.getOperator();
+	}
+
+	@Override
+	protected List<OperatorDescriptorDual> getPossibleProperties() {
+		return dataProperties;
+	}
+
+	@Override
+	public String getOperatorName() {
+		return "Outer Join";
+	}
+
+	@Override
+	protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) {
+		long card1 = getFirstPredecessorNode().getEstimatedNumRecords();
+		long card2 = getSecondPredecessorNode().getEstimatedNumRecords();
+
+		if (card1 < 0 || card2 < 0) {
+			this.estimatedNumRecords = -1;
+		} else {
+			this.estimatedNumRecords = Math.max(card1, card2);
+		}
+
+		if (this.estimatedNumRecords >= 0) {
+			float width1 = getFirstPredecessorNode().getEstimatedAvgWidthPerOutputRecord();
+			float width2 = getSecondPredecessorNode().getEstimatedAvgWidthPerOutputRecord();
+			float width = (width1 <= 0 || width2 <= 0) ? -1 : width1 + width2;
+
+			if (width > 0) {
+				this.estimatedOutputSize = (long) (width * this.estimatedNumRecords);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java
new file mode 100644
index 0000000..d54b5cf
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java
@@ -0,0 +1,81 @@
+/*
+ * 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.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.optimizer.CompilerException;
+import org.apache.flink.optimizer.dag.TwoInputNode;
+import org.apache.flink.optimizer.dataproperties.LocalProperties;
+import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties;
+import org.apache.flink.optimizer.plan.Channel;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.util.Utils;
+
+import java.util.Collections;
+import java.util.List;
+
+public abstract class AbstractSortMergeJoinDescriptor extends AbstractJoinDescriptor {
+
+	public AbstractSortMergeJoinDescriptor(FieldList keys1, FieldList keys2) {
+		super(keys1, keys2);
+	}
+
+	public AbstractSortMergeJoinDescriptor(FieldList keys1, FieldList keys2,
+			boolean broadcastFirstAllowed, boolean broadcastSecondAllowed, boolean repartitionAllowed) {
+		super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed);
+	}
+
+	@Override
+	protected List<LocalPropertiesPair> createPossibleLocalProperties() {
+		RequestedLocalProperties sort1 = new RequestedLocalProperties(Utils.createOrdering(this.keys1));
+		RequestedLocalProperties sort2 = new RequestedLocalProperties(Utils.createOrdering(this.keys2));
+		return Collections.singletonList(new LocalPropertiesPair(sort1, sort2));
+	}
+
+	@Override
+	public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2,
+			LocalProperties produced1, LocalProperties produced2) {
+		int numRelevantFields = this.keys1.size();
+		return checkSameOrdering(produced1, produced2, numRelevantFields);
+	}
+
+	@Override
+	public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) {
+		boolean[] inputOrders = in1.getLocalProperties().getOrdering().getFieldSortDirections();
+
+		if (inputOrders == null || inputOrders.length < this.keys1.size()) {
+			throw new CompilerException("BUG: The input strategy does not sufficiently describe the sort orders for a merge operator.");
+		} else if (inputOrders.length > this.keys1.size()) {
+			boolean[] tmp = new boolean[this.keys1.size()];
+			System.arraycopy(inputOrders, 0, tmp, 0, tmp.length);
+			inputOrders = tmp;
+		}
+
+		String nodeName = String.format("%s(%s)", getNodeName(), node.getOperator().getName());
+		return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2, inputOrders);
+	}
+
+	@Override
+	public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) {
+		LocalProperties comb = LocalProperties.combine(in1, in2);
+		return comb.clearUniqueFieldSets();
+	}
+
+	protected abstract String getNodeName();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java
index 368944e..571d6e6 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java
@@ -177,30 +177,9 @@ public class CoGroupDescriptor extends OperatorDescriptorDual {
 	
 	@Override
 	public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2,
-			LocalProperties produced1, LocalProperties produced2)
-	{
+			LocalProperties produced1, LocalProperties produced2) {
 		int numRelevantFields = this.keys1.size();
-		
-		Ordering prod1 = produced1.getOrdering();
-		Ordering prod2 = produced2.getOrdering();
-		
-		if (prod1 == null || prod2 == null) {
-			throw new CompilerException("The given properties do not meet this operators requirements.");
-		}
-
-		// check that order of fields is equivalent
-		if (!checkEquivalentFieldPositionsInKeyFields(
-				prod1.getInvolvedIndexes(), prod2.getInvolvedIndexes(), numRelevantFields)) {
-			return false;
-		}
-
-		// check that order directions are equivalent
-		for (int i = 0; i < numRelevantFields; i++) {
-			if (prod1.getOrder(i) != prod2.getOrder(i)) {
-				return false;
-			}
-		}
-		return true;
+		return checkSameOrdering(produced1, produced2, numRelevantFields);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java
index c21593e..17ea8a5 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java
@@ -21,6 +21,7 @@ package org.apache.flink.optimizer.operators;
 
 import java.util.List;
 
+import org.apache.flink.api.common.operators.Ordering;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.optimizer.dag.TwoInputNode;
@@ -124,6 +125,29 @@ public abstract class OperatorDescriptorDual implements AbstractOperatorDescript
 		return true;
 	}
 
+	protected boolean checkSameOrdering(LocalProperties produced1, LocalProperties produced2, int numRelevantFields) {
+		Ordering prod1 = produced1.getOrdering();
+		Ordering prod2 = produced2.getOrdering();
+
+		if (prod1 == null || prod2 == null) {
+			throw new CompilerException("The given properties do not meet this operators requirements.");
+		}
+
+		// check that order of fields is equivalent
+		if (!checkEquivalentFieldPositionsInKeyFields(
+				prod1.getInvolvedIndexes(), prod2.getInvolvedIndexes(), numRelevantFields)) {
+			return false;
+		}
+
+		// check that both inputs have the same directions of order
+		for (int i = 0; i < numRelevantFields; i++) {
+			if (prod1.getOrder(i) != prod2.getOrder(i)) {
+				return false;
+			}
+		}
+		return true;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	
 	public static final class GlobalPropertiesPair {

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java
new file mode 100644
index 0000000..4e05067
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.runtime.operators.DriverStrategy;
+
+public class SortMergeFullOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor {
+
+	public SortMergeFullOuterJoinDescriptor(FieldList keys1, FieldList keys2) {
+		super(keys1, keys2, false, false, true);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.FULL_OUTER_MERGE;
+	}
+
+	@Override
+	protected String getNodeName() {
+		return "FullOuterJoin";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java
new file mode 100644
index 0000000..1c3ea19
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.runtime.operators.DriverStrategy;
+
+public class SortMergeInnerJoinDescriptor extends AbstractSortMergeJoinDescriptor {
+
+	public SortMergeInnerJoinDescriptor(FieldList keys1, FieldList keys2) {
+		super(keys1, keys2);
+	}
+
+	public SortMergeInnerJoinDescriptor(FieldList keys1, FieldList keys2,
+			boolean broadcastFirstAllowed, boolean broadcastSecondAllowed, boolean repartitionAllowed) {
+		super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.INNER_MERGE;
+	}
+
+	@Override
+	protected String getNodeName() {
+		return "Join";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeJoinDescriptor.java
deleted file mode 100644
index 3ab0aa7..0000000
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeJoinDescriptor.java
+++ /dev/null
@@ -1,110 +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.optimizer.operators;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.operators.Ordering;
-import org.apache.flink.api.common.operators.util.FieldList;
-import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.optimizer.dag.TwoInputNode;
-import org.apache.flink.optimizer.dataproperties.LocalProperties;
-import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties;
-import org.apache.flink.optimizer.plan.Channel;
-import org.apache.flink.optimizer.plan.DualInputPlanNode;
-import org.apache.flink.optimizer.util.Utils;
-import org.apache.flink.runtime.operators.DriverStrategy;
-
-/**
- * 
- */
-public class SortMergeJoinDescriptor extends AbstractJoinDescriptor {
-	
-	public SortMergeJoinDescriptor(FieldList keys1, FieldList keys2) {
-		super(keys1, keys2);
-	}
-	
-	public SortMergeJoinDescriptor(FieldList keys1, FieldList keys2,
-			boolean broadcastFirstAllowed, boolean broadcastSecondAllowed, boolean repartitionAllowed)
-	{
-		super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed);
-	}
-
-	@Override
-	public DriverStrategy getStrategy() {
-		return DriverStrategy.INNER_MERGE;
-	}
-
-	@Override
-	protected List<LocalPropertiesPair> createPossibleLocalProperties() {
-		RequestedLocalProperties sort1 = new RequestedLocalProperties(Utils.createOrdering(this.keys1));
-		RequestedLocalProperties sort2 = new RequestedLocalProperties(Utils.createOrdering(this.keys2));
-		return Collections.singletonList(new LocalPropertiesPair(sort1, sort2));
-	}
-
-	@Override
-	public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2,
-			LocalProperties produced1, LocalProperties produced2)
-	{
-		int numRelevantFields = this.keys1.size();
-		
-		Ordering prod1 = produced1.getOrdering();
-		Ordering prod2 = produced2.getOrdering();
-		
-		if (prod1 == null || prod2 == null) {
-			throw new CompilerException("The given properties do not meet this operators requirements.");
-		}
-
-		// check that order of fields is equivalent
-		if (!checkEquivalentFieldPositionsInKeyFields(
-				prod1.getInvolvedIndexes(), prod2.getInvolvedIndexes(), numRelevantFields)) {
-			return false;
-		}
-
-		// check that both inputs have the same directions of order
-		for (int i = 0; i < numRelevantFields; i++) {
-			if (prod1.getOrder(i) != prod2.getOrder(i)) {
-				return false;
-			}
-		}
-		return true;
-	}
-	
-	@Override
-	public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) {
-		boolean[] inputOrders = in1.getLocalProperties().getOrdering().getFieldSortDirections();
-		
-		if (inputOrders == null || inputOrders.length < this.keys1.size()) {
-			throw new CompilerException("BUG: The input strategy does not sufficiently describe the sort orders for a merge operator.");
-		} else if (inputOrders.length > this.keys1.size()) {
-			boolean[] tmp = new boolean[this.keys1.size()];
-			System.arraycopy(inputOrders, 0, tmp, 0, tmp.length);
-			inputOrders = tmp;
-		}
-		
-		return new DualInputPlanNode(node, "Join(" + node.getOperator().getName() + ")", in1, in2, DriverStrategy.INNER_MERGE, this.keys1, this.keys2, inputOrders);
-	}
-
-	@Override
-	public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) {
-		LocalProperties comb = LocalProperties.combine(in1, in2);
-		return comb.clearUniqueFieldSets();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java
new file mode 100644
index 0000000..8193960
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java
@@ -0,0 +1,43 @@
+/*
+ * 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.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.runtime.operators.DriverStrategy;
+
+public class SortMergeLeftOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor {
+
+	public SortMergeLeftOuterJoinDescriptor(FieldList keys1, FieldList keys2) {
+		super(keys1, keys2, false, true, true);
+	}
+
+	public SortMergeLeftOuterJoinDescriptor(FieldList keys1, FieldList keys2, boolean broadcastAllowed) {
+		super(keys1, keys2, false, broadcastAllowed, true);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.LEFT_OUTER_MERGE;
+	}
+
+	@Override
+	protected String getNodeName() {
+		return "LeftOuterJoin";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java
new file mode 100644
index 0000000..3719d05
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java
@@ -0,0 +1,43 @@
+/*
+ * 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.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.runtime.operators.DriverStrategy;
+
+public class SortMergeRightOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor {
+
+	public SortMergeRightOuterJoinDescriptor(FieldList keys1, FieldList keys2) {
+		super(keys1, keys2, true, false, true);
+	}
+
+	public SortMergeRightOuterJoinDescriptor(FieldList keys1, FieldList keys2, boolean broadcastAllowed) {
+		super(keys1, keys2, broadcastAllowed, false, true);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.RIGHT_OUTER_MERGE;
+	}
+
+	@Override
+	protected String getNodeName() {
+		return "RightOuterJoin";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
index 7fbdf81..bcdee14 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
@@ -26,15 +26,17 @@ import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.Union;
 import org.apache.flink.api.common.operators.base.BulkIterationBase;
 import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
+import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase;
 import org.apache.flink.api.common.operators.base.CrossOperatorBase;
 import org.apache.flink.api.common.operators.base.DeltaIterationBase;
 import org.apache.flink.api.common.operators.base.FilterOperatorBase;
 import org.apache.flink.api.common.operators.base.FlatMapOperatorBase;
 import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase;
 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.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase;
+import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.PartitionOperatorBase;
 import org.apache.flink.api.common.operators.base.ReduceOperatorBase;
 import org.apache.flink.api.common.operators.base.SortPartitionOperatorBase;
@@ -44,6 +46,7 @@ import org.apache.flink.optimizer.dag.BinaryUnionNode;
 import org.apache.flink.optimizer.dag.BulkIterationNode;
 import org.apache.flink.optimizer.dag.BulkPartialSolutionNode;
 import org.apache.flink.optimizer.dag.CoGroupNode;
+import org.apache.flink.optimizer.dag.CoGroupRawNode;
 import org.apache.flink.optimizer.dag.CollectorMapNode;
 import org.apache.flink.optimizer.dag.CrossNode;
 import org.apache.flink.optimizer.dag.DagConnection;
@@ -57,6 +60,7 @@ import org.apache.flink.optimizer.dag.JoinNode;
 import org.apache.flink.optimizer.dag.MapNode;
 import org.apache.flink.optimizer.dag.MapPartitionNode;
 import org.apache.flink.optimizer.dag.OptimizerNode;
+import org.apache.flink.optimizer.dag.OuterJoinNode;
 import org.apache.flink.optimizer.dag.PartitionNode;
 import org.apache.flink.optimizer.dag.ReduceNode;
 import org.apache.flink.optimizer.dag.SolutionSetNode;
@@ -69,8 +73,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase;
-import org.apache.flink.optimizer.dag.CoGroupRawNode;
 
 /**
  * This traversal creates the optimizer DAG from a program.
@@ -160,8 +162,11 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 		else if (c instanceof GroupReduceOperatorBase) {
 			n = new GroupReduceNode((GroupReduceOperatorBase<?, ?, ?>) c);
 		}
-		else if (c instanceof JoinOperatorBase) {
-			n = new JoinNode((JoinOperatorBase<?, ?, ?, ?>) c);
+		else if (c instanceof InnerJoinOperatorBase) {
+			n = new JoinNode((InnerJoinOperatorBase<?, ?, ?, ?>) c);
+		}
+		else if (c instanceof OuterJoinOperatorBase) {
+			n = new OuterJoinNode((OuterJoinOperatorBase<?, ?, ?, ?>) c);
 		}
 		else if (c instanceof CoGroupOperatorBase) {
 			n = new CoGroupNode((CoGroupOperatorBase<?, ?, ?, ?>) c);

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java
index 8c19462..17f0241 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.operators.OperatorInformation;
 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.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.common.operators.util.FieldSet;
@@ -1431,6 +1431,6 @@ public class FeedbackPropertiesMatchTest {
 	}
 	
 	private static JoinNode getJoinNode() {
-		return new JoinNode(new JoinOperatorBase<String, String, String, FlatJoinFunction<String, String, String>>(new DummyFlatJoinFunction<String>(), new BinaryOperatorInformation<String, String, String>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op"));
+		return new JoinNode(new InnerJoinOperatorBase<String, String, String, FlatJoinFunction<String, String, String>>(new DummyFlatJoinFunction<String>(), new BinaryOperatorInformation<String, String, String>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op"));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
index 00ada2a..321e5ca 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
@@ -21,7 +21,7 @@ package org.apache.flink.optimizer;
 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.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.common.operators.base.ReduceOperatorBase;
 import org.apache.flink.api.common.operators.util.FieldSet;
@@ -126,7 +126,7 @@ public class SemanticPropertiesAPIToPlanTest extends CompilerTestBase {
 		oPlan.accept(new Visitor<PlanNode>() {
 			@Override
 			public boolean preVisit(PlanNode visitable) {
-				if (visitable instanceof DualInputPlanNode && visitable.getProgramOperator() instanceof JoinOperatorBase) {
+				if (visitable instanceof DualInputPlanNode && visitable.getProgramOperator() instanceof InnerJoinOperatorBase) {
 					DualInputPlanNode node = ((DualInputPlanNode) visitable);
 
 					final Channel inConn1 = node.getInput1();

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java
index 839f0a1..1d559c2 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java
@@ -34,8 +34,8 @@ public class JoinGlobalPropertiesCompatibilityTest {
 		try {
 			final FieldList keysLeft = new FieldList(1, 4);
 			final FieldList keysRight = new FieldList(3, 1);
-			
-			SortMergeJoinDescriptor descr = new SortMergeJoinDescriptor(keysLeft, keysRight);
+
+			SortMergeInnerJoinDescriptor descr = new SortMergeInnerJoinDescriptor(keysLeft, keysRight);
 			
 			// test compatible hash partitioning
 			{
@@ -121,7 +121,7 @@ public class JoinGlobalPropertiesCompatibilityTest {
 				}
 			};
 			
-			SortMergeJoinDescriptor descr = new SortMergeJoinDescriptor(keysLeft, keysRight);
+			SortMergeInnerJoinDescriptor descr = new SortMergeInnerJoinDescriptor(keysLeft, keysRight);
 			
 			// test incompatible hash with custom partitioning
 			{

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 207bc5d..306a15b 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.aggregators.Aggregator
 import org.apache.flink.api.common.functions._
 import org.apache.flink.api.common.io.{FileOutputFormat, OutputFormat}
 import org.apache.flink.api.common.operators.Order
-import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
 import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
+import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
 import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.Utils.CountHelper
@@ -33,6 +33,7 @@ import org.apache.flink.api.java.functions.{FirstReducer, KeySelector}
 import org.apache.flink.api.java.io.{DiscardingOutputFormat, PrintingOutputFormat, TextOutputFormat}
 import org.apache.flink.api.java.operators.Keys.ExpressionKeys
 import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.operators.join.JoinType
 import org.apache.flink.api.java.{DataSet => JavaDataSet, Utils}
 import org.apache.flink.api.scala.operators.{ScalaAggregateOperator, ScalaCsvOutputFormat}
 import org.apache.flink.configuration.Configuration
@@ -840,11 +841,11 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
 
   /**
    * Creates a new DataSet by joining `this` DataSet with the `other` DataSet. To specify the join
-   * keys the `where` and `isEqualTo` methods must be used. For example:
+   * keys the `where` and `equalTo` methods must be used. For example:
    * {{{
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...
-   *   val joined = left.join(right).where(0).isEqualTo(1)
+   *   val joined = left.join(right).where(0).equalTo(1)
    * }}}
    *
    * The default join result is a DataSet with 2-Tuples of the joined values. In the above example
@@ -854,7 +855,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    * {{{
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...
-   *   val joined = left.join(right).where(0).isEqualTo(1) { (l, r) =>
+   *   val joined = left.join(right).where(0).equalTo(1) { (l, r) =>
    *     (l._1, r._2)
    *   }
    * }}}
@@ -864,7 +865,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    * {{{
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...
-   *   val joined = left.join(right).where(0).isEqualTo(1) {
+   *   val joined = left.join(right).where(0).equalTo(1) {
    *     (l, r, out: Collector[(String, Int)]) =>
    *       if (l._2 > 4) {
    *         out.collect((l._1, r._3))
@@ -875,29 +876,119 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    *     }
    * }}}
    */
-  def join[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES)
+  def join[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
+    new UnfinishedInnerJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES)
 
   /**
    * Special [[join]] operation for explicitly telling the system what join strategy to use. If
    * null is given as the join strategy, then the optimizer will pick the strategy.
    */
-  def join[O](other: DataSet[O], strategy: JoinHint): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperation(this, other, strategy)
+  def join[O](other: DataSet[O], strategy: JoinHint): UnfinishedInnerJoinOperation[T, O] =
+    new UnfinishedInnerJoinOperation(this, other, strategy)
   
   /**
    * Special [[join]] operation for explicitly telling the system that the right side is assumed
    * to be a lot smaller than the left side of the join.
    */
-  def joinWithTiny[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperation(this, other, JoinHint.BROADCAST_HASH_SECOND)
+  def joinWithTiny[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
+    new UnfinishedInnerJoinOperation(this, other, JoinHint.BROADCAST_HASH_SECOND)
 
   /**
    * Special [[join]] operation for explicitly telling the system that the left side is assumed
    * to be a lot smaller than the right side of the join.
    */
-  def joinWithHuge[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperation(this, other, JoinHint.BROADCAST_HASH_FIRST)
+  def joinWithHuge[O](other: DataSet[O]): UnfinishedInnerJoinOperation[T, O] =
+    new UnfinishedInnerJoinOperation(this, other, JoinHint.BROADCAST_HASH_FIRST)
+
+  /**
+   * Creates a new DataSet by performing a full outer join of `this` DataSet
+   * with the `other` DataSet, by combining two elements of two DataSets on
+   * key equality.
+   * Elements of both DataSets that do not have a matching element on the
+   * opposing side are joined with `null` and emitted to the resulting DataSet.
+   *
+   * To specify the join keys the `where` and `equalTo` methods must be used. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val joined = left.fullOuterJoin(right).where(0).equalTo(1)
+   * }}}
+   *
+   * When using an outer join you are required to specify a join function. For example:
+   * {{{
+   *   val joined = left.fullOuterJoin(right).where(0).equalTo(1) {
+   *     (left, right) =>
+   *       val a = if (left == null) null else left._1
+   *       val b = if (right == null) null else right._3
+   *       (a, b)
+   *  }
+   * }}}
+   */
+  def fullOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.FULL_OUTER)
+
+  /**
+   * Special [[fullOuterJoin]] operation for explicitly telling the system what join strategy to
+   * use. If null is given as the join strategy, then the optimizer will pick the strategy.
+   */
+  def fullOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.FULL_OUTER)
+
+  /**
+   * An outer join on the left side.
+   *
+   * Elements of the left side (i.e. `this`) that do not have a matching element on the other
+   * side are joined with `null` and emitted to the resulting DataSet.
+   *
+   * @param other The other DataSet with which this DataSet is joined.
+   * @return An UnfinishedJoinOperation to continue with the definition of the join transformation
+   * @see #fullOuterJoin
+   */
+  def leftOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.LEFT_OUTER)
+
+  /**
+   * An outer join on the left side.
+   *
+   * Elements of the left side (i.e. `this`) that do not have a matching element on the other
+   * side are joined with `null` and emitted to the resulting DataSet.
+   *
+   * @param other The other DataSet with which this DataSet is joined.
+   * @param strategy The strategy that should be used execute the join. If { @code null} is given,
+   *                 then the optimizer will pick the join strategy.
+   * @return An UnfinishedJoinOperation to continue with the definition of the join transformation
+   * @see #fullOuterJoin
+   */
+  def leftOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.LEFT_OUTER)
+
+  /**
+   * An outer join on the right side.
+   *
+   * Elements of the right side (i.e. `other`) that do not have a matching element on `this`
+   * side are joined with `null` and emitted to the resulting DataSet.
+   *
+   * @param other The other DataSet with which this DataSet is joined.
+   * @return An UnfinishedJoinOperation to continue with the definition of the join transformation
+   * @see #fullOuterJoin
+   */
+  def rightOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, JoinHint.OPTIMIZER_CHOOSES, JoinType.RIGHT_OUTER)
+
+  /**
+   * An outer join on the right side.
+   *
+   * Elements of the right side (i.e. `other`) that do not have a matching element on `this`
+   * side are joined with `null` and emitted to the resulting DataSet.
+   *
+   * @param other The other DataSet with which this DataSet is joined.
+   * @param strategy The strategy that should be used execute the join. If { @code null} is given,
+   *                 then the optimizer will pick the join strategy.
+   * @return An UnfinishedJoinOperation to continue with the definition of the join transformation
+   * @see #fullOuterJoin
+   */
+  def rightOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O] =
+    new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.RIGHT_OUTER)
 
   // --------------------------------------------------------------------------------------------
   //  Co-Group

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index ecc1aab..f57fc25 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -17,12 +17,15 @@
  */
 package org.apache.flink.api.scala
 
+import org.apache.flink.api.common.operators.Operator
+import org.apache.flink.api.common.operators.base.JoinOperatorBase
 import org.apache.flink.api.common.functions.{FlatJoinFunction, JoinFunction, Partitioner, RichFlatJoinFunction}
 import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin.WrappingFlatJoinFunction
 import org.apache.flink.api.java.operators.JoinOperator.EquiJoin
 import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.operators.join.JoinType
 import org.apache.flink.util.Collector
 
 import scala.reflect.ClassTag
@@ -60,7 +63,7 @@ class JoinDataSet[L, R](
     rightInput: DataSet[R],
     leftKeys: Keys[L],
     rightKeys: Keys[R])
-  extends DataSet(defaultJoin) {
+  extends DataSet(defaultJoin) with JoinFunctionAssigner[L, R] {
 
   private var customPartitioner : Partitioner[_] = _
   
@@ -84,7 +87,8 @@ class JoinDataSet[L, R](
       joiner,
       implicitly[TypeInformation[O]],
       defaultJoin.getJoinHint,
-      getCallLocationName())
+      getCallLocationName(),
+      defaultJoin.getJoinType)
     
     if (customPartitioner != null) {
       wrap(joinOperator.withPartitioner(customPartitioner))
@@ -114,7 +118,8 @@ class JoinDataSet[L, R](
       joiner,
       implicitly[TypeInformation[O]],
       defaultJoin.getJoinHint,
-      getCallLocationName())
+      getCallLocationName(),
+      defaultJoin.getJoinType)
 
     if (customPartitioner != null) {
       wrap(joinOperator.withPartitioner(customPartitioner))
@@ -142,7 +147,8 @@ class JoinDataSet[L, R](
       joiner,
       implicitly[TypeInformation[O]],
       defaultJoin.getJoinHint,
-      getCallLocationName())
+      getCallLocationName(),
+      defaultJoin.getJoinType)
 
     if (customPartitioner != null) {
       wrap(joinOperator.withPartitioner(customPartitioner))
@@ -171,7 +177,8 @@ class JoinDataSet[L, R](
       generatedFunction, fun,
       implicitly[TypeInformation[O]],
       defaultJoin.getJoinHint,
-      getCallLocationName())
+      getCallLocationName(),
+      defaultJoin.getJoinType)
 
     if (customPartitioner != null) {
       wrap(joinOperator.withPartitioner(customPartitioner))
@@ -205,9 +212,46 @@ class JoinDataSet[L, R](
   }
 }
 
+private[flink] abstract class UnfinishedJoinOperation[L, R, O <: JoinFunctionAssigner[L, R]](
+    leftSet: DataSet[L],
+    rightSet: DataSet[R],
+    val joinHint: JoinHint,
+    val joinType: JoinType)
+  extends UnfinishedKeyPairOperation[L, R, O](leftSet, rightSet) {
+
+  def createJoinFunctionAssigner(leftKey: Keys[L], rightKey: Keys[R]): O
+
+  private[flink] def createDefaultJoin(leftKey: Keys[L], rightKey: Keys[R]) = {
+    val joiner = new FlatJoinFunction[L, R, (L, R)] {
+      def join(left: L, right: R, out: Collector[(L, R)]) = {
+        out.collect((left, right))
+      }
+    }
+    val returnType = createTuple2TypeInformation[L, R](leftInput.getType(), rightInput.getType())
+    val joinOperator = new EquiJoin[L, R, (L, R)](
+      leftSet.javaSet,
+      rightSet.javaSet,
+      leftKey,
+      rightKey,
+      joiner,
+      returnType,
+      joinHint,
+      getCallLocationName(),
+      joinType)
+
+    new JoinDataSet(joinOperator, leftSet, rightSet, leftKey, rightKey)
+  }
+
+  private[flink] def finish(leftKey: Keys[L], rightKey: Keys[R]) = {
+    createJoinFunctionAssigner(leftKey, rightKey)
+  }
+}
+
 /**
- * An unfinished join operation that results from [[DataSet.join()]] The keys for the left and right
- * side must be specified using first `where` and then `equalTo`. For example:
+ * An unfinished inner join operation that results from calling [[DataSet.join()]].
+ * The keys for the left and right side must be specified using first `where` and then `equalTo`.
+ *
+ * For example:
  *
  * {{{
  *   val left = ...
@@ -217,24 +261,77 @@ class JoinDataSet[L, R](
  * @tparam L The type of the left input of the join.
  * @tparam R The type of the right input of the join.
  */
-class UnfinishedJoinOperation[L, R](
+class UnfinishedInnerJoinOperation[L, R](
     leftSet: DataSet[L],
     rightSet: DataSet[R],
-    val joinHint: JoinHint)
-  extends UnfinishedKeyPairOperation[L, R, JoinDataSet[L, R]](leftSet, rightSet) {
+    joinHint: JoinHint)
+  extends UnfinishedJoinOperation[L, R, JoinDataSet[L, R]](
+    leftSet, rightSet, joinHint, JoinType.INNER) {
 
-  private[flink] def finish(leftKey: Keys[L], rightKey: Keys[R]) = {
-    val joiner = new FlatJoinFunction[L, R, (L, R)] {
-      def join(left: L, right: R, out: Collector[(L, R)]) = {
-        out.collect((left, right))
-      }
-    }
-    val returnType = createTuple2TypeInformation[L, R](leftInput.getType(), rightInput.getType())
-    val joinOperator = new EquiJoin[L, R, (L, R)](
-      leftSet.javaSet, rightSet.javaSet, leftKey, rightKey, joiner, returnType, joinHint,
-        getCallLocationName())
+  override def createJoinFunctionAssigner(leftKey: Keys[L], rightKey: Keys[R]) = {
+    createDefaultJoin(leftKey, rightKey)
+  }
+}
 
-    new JoinDataSet(joinOperator, leftSet, rightSet, leftKey, rightKey)
+/**
+ * An unfinished outer join operation that results from calling, e.g. [[DataSet.fullOuterJoin()]].
+ * The keys for the left and right side must be specified using first `where` and then `equalTo`.
+ *
+ * Note that a join function must always be specified explicitly when construction an outer join
+ * operator.
+ *
+ * For example:
+ *
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val joinResult = left.fullOuterJoin(right).where(...).equalTo(...) {
+ *     (first, second) => ...
+ *   }
+ * }}}
+ * @tparam L The type of the left input of the join.
+ * @tparam R The type of the right input of the join.
+ */
+class UnfinishedOuterJoinOperation[L, R](
+    leftSet: DataSet[L],
+    rightSet: DataSet[R],
+    joinHint: JoinHint,
+    joinType: JoinType)
+  extends UnfinishedJoinOperation[L, R, JoinFunctionAssigner[L, R]](
+    leftSet, rightSet, joinHint, joinType) {
+
+  override def createJoinFunctionAssigner(leftKey: Keys[L], rightKey: Keys[R]):
+      JoinFunctionAssigner[L, R] = {
+    new DefaultJoinFunctionAssigner(createDefaultJoin(leftKey, rightKey))
+  }
+
+  private class DefaultJoinFunctionAssigner(val defaultJoin: JoinDataSet[L, R])
+    extends JoinFunctionAssigner[L, R] {
+
+    override def withPartitioner[K: TypeInformation](part: Partitioner[K]) =
+      defaultJoin.withPartitioner(part)
+
+    override def apply[O: TypeInformation : ClassTag](fun: (L, R) => O) =
+      defaultJoin.apply(fun)
+
+    override def apply[O: TypeInformation : ClassTag](fun: (L, R, Collector[O]) => Unit) =
+      defaultJoin.apply(fun)
+
+    override def apply[O: TypeInformation : ClassTag](fun: FlatJoinFunction[L, R, O]) =
+      defaultJoin.apply(fun)
+
+    override def apply[O: TypeInformation : ClassTag](fun: JoinFunction[L, R, O]) =
+      defaultJoin.apply(fun)
   }
 
 }
+
+trait JoinFunctionAssigner[L, R] {
+
+  def withPartitioner[K : TypeInformation](part : Partitioner[K]) : JoinFunctionAssigner[L, R]
+  def apply[O: TypeInformation: ClassTag](fun: (L, R) => O): DataSet[O]
+  def apply[O: TypeInformation: ClassTag](fun: (L, R, Collector[O]) => Unit): DataSet[O]
+  def apply[O: TypeInformation: ClassTag](fun: FlatJoinFunction[L, R, O]): DataSet[O]
+  def apply[O: TypeInformation: ClassTag](fun: JoinFunction[L, R, O]): DataSet[O]
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b00c1d7e/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index 566573e..7605b3a 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -29,7 +29,6 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.junit.Assert;
 
 public class TestEnvironment extends ExecutionEnvironment {
 
@@ -60,21 +59,13 @@ public class TestEnvironment extends ExecutionEnvironment {
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		try {
-			OptimizedPlan op = compileProgram(jobName);
+		OptimizedPlan op = compileProgram(jobName);
 
-			JobGraphGenerator jgg = new JobGraphGenerator();
-			JobGraph jobGraph = jgg.compileJobGraph(op);
-			
-			this.lastJobExecutionResult = executor.submitJobAndWait(jobGraph, false);
-			return this.lastJobExecutionResult;
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Job execution failed!");
-			return null;
-		}
+		JobGraphGenerator jgg = new JobGraphGenerator();
+		JobGraph jobGraph = jgg.compileJobGraph(op);
+
+		this.lastJobExecutionResult = executor.submitJobAndWait(jobGraph, false);
+		return this.lastJobExecutionResult;
 	}