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/19 18:01:07 UTC

[01/10] flink git commit: [FLINK-2809] [scala-api] Added UnitTypeInfo and UnitSerializer.

Repository: flink
Updated Branches:
  refs/heads/master 8c1719c88 -> 5671c77c3


[FLINK-2809] [scala-api] Added UnitTypeInfo and UnitSerializer.

This closes #1217


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

Branch: refs/heads/master
Commit: 64915590168240d238259c249a418b7dcafc50d3
Parents: 8c1719c
Author: Gabor Gevay <gg...@gmail.com>
Authored: Fri Oct 2 18:04:59 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 19 15:38:31 2015 +0200

----------------------------------------------------------------------
 .../api/java/io/CollectionInputFormat.java      | 11 +++-
 .../flink/api/scala/codegen/TypeAnalyzer.scala  |  8 ++-
 .../api/scala/codegen/TypeDescriptors.scala     |  4 +-
 .../api/scala/codegen/TypeInformationGen.scala  |  6 +-
 .../api/scala/typeutils/UnitSerializer.scala    | 58 ++++++++++++++++++++
 .../api/scala/typeutils/UnitTypeInfo.scala      | 46 ++++++++++++++++
 .../scala/types/TypeInformationGenTest.scala    | 14 ++++-
 7 files changed, 140 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java
index 97e8715..1df995b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java
@@ -130,8 +130,15 @@ public class CollectionInputFormat<T> extends GenericInputFormat<T> implements N
 			if (elem == null) {
 				throw new IllegalArgumentException("The collection must not contain null elements.");
 			}
-			
-			if (!viewedAs.isAssignableFrom(elem.getClass())) {
+
+			// The second part of the condition is a workaround for the situation that can arise from eg.
+			// "env.fromElements((),(),())"
+			// In this situation, UnitTypeInfo.getTypeClass returns void.class (when we are in the Java world), but
+			// the actual objects that we will be working with, will be BoxedUnits.
+			// Note: TypeInformationGenTest.testUnit tests this condition.
+			if (!viewedAs.isAssignableFrom(elem.getClass()) &&
+					!(elem.getClass().toString().equals("class scala.runtime.BoxedUnit") && viewedAs.equals(void.class))) {
+
 				throw new IllegalArgumentException("The elements in the collection are not all subclasses of " + 
 							viewedAs.getCanonicalName());
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
index 0421557..4d0d353 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
@@ -56,7 +56,9 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
 
           case ArrayType(elemTpe) => analyzeArray(id, tpe, elemTpe)
 
-          case NothingType() => NothingDesciptor(id, tpe)
+          case NothingType() => NothingDescriptor(id, tpe)
+
+          case UnitType() => UnitDescriptor(id, tpe)
 
           case EitherType(leftTpe, rightTpe) => analyzeEither(id, tpe, leftTpe, rightTpe)
 
@@ -339,6 +341,10 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
       def unapply(tpe: Type): Boolean = tpe =:= typeOf[Nothing]
     }
 
+    private object UnitType {
+      def unapply(tpe: Type): Boolean = tpe =:= typeOf[Unit]
+    }
+
     private object EitherType {
       def unapply(tpe: Type): Option[(Type, Type)] = {
         if (tpe <:< typeOf[Either[_, _]]) {

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
index b65defd..aea39bf 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
@@ -39,7 +39,9 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
   case class PrimitiveDescriptor(id: Int, tpe: Type, default: Literal, wrapper: Type)
     extends UDTDescriptor
 
-  case class NothingDesciptor(id: Int, tpe: Type) extends UDTDescriptor
+  case class NothingDescriptor(id: Int, tpe: Type) extends UDTDescriptor
+
+  case class UnitDescriptor(id: Int, tpe: Type) extends UDTDescriptor
 
   case class EitherDescriptor(id: Int, tpe: Type, left: UDTDescriptor, right: UDTDescriptor)
     extends UDTDescriptor

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 499bc5b..e30cb6d 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo._
 import org.apache.flink.api.common.typeutils._
 import org.apache.flink.api.java.typeutils._
-import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo, ScalaNothingTypeInfo}
+import org.apache.flink.api.scala.typeutils._
 import org.apache.flink.types.Value
 import org.apache.hadoop.io.Writable
 
@@ -59,9 +59,11 @@ private[flink] trait TypeInformationGen[C <: Context] {
     case p : PrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
     case p : BoxedPrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
 
-    case n: NothingDesciptor =>
+    case n: NothingDescriptor =>
       reify { new ScalaNothingTypeInfo().asInstanceOf[TypeInformation[T]] }
 
+    case u: UnitDescriptor => reify { new UnitTypeInfo().asInstanceOf[TypeInformation[T]] }
+
     case e: EitherDescriptor => mkEitherTypeInfo(e)
 
     case e: EnumValueDescriptor => mkEnumValueTypeInfo(e)

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
new file mode 100644
index 0000000..bb526ba
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton
+import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+
+class UnitSerializer extends TypeSerializerSingleton[Unit] {
+
+  def isImmutableType(): Boolean = true
+
+  def createInstance(): Unit = ()
+
+  def copy(from: Unit): Unit = ()
+
+  def copy(from: Unit, reuse: Unit): Unit = ()
+
+  def getLength(): Int = 1
+
+  def serialize(record: Unit, target: DataOutputView) {
+    target.write(0)
+  }
+
+  def deserialize(source: DataInputView): Unit = {
+    source.readByte()
+    ()
+  }
+
+  def deserialize(reuse: Unit, source: DataInputView): Unit = {
+    source.readByte()
+    ()
+  }
+
+  def copy(source: DataInputView, target: DataOutputView) {
+    target.write(source.readByte)
+  }
+
+  override def hashCode(): Int = classOf[UnitSerializer].hashCode
+
+  override def canEqual(obj: scala.Any): Boolean = {
+    obj.isInstanceOf[UnitSerializer]
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfo.scala
new file mode 100644
index 0000000..e0a226b
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfo.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeutils.TypeSerializer
+
+class UnitTypeInfo extends TypeInformation[Unit] {
+  override def isBasicType(): Boolean = false
+  override def isTupleType(): Boolean = false
+  override def getArity(): Int = 0
+  override def getTotalFields(): Int = 0
+  override def getTypeClass(): Class[Unit] = classOf[Unit]
+  override def isKeyType(): Boolean = false
+
+  override def createSerializer(config: ExecutionConfig): TypeSerializer[Unit] =
+    (new UnitSerializer).asInstanceOf[TypeSerializer[Unit]]
+
+  override def canEqual(obj: scala.Any): Boolean = {
+    obj.isInstanceOf[UnitTypeInfo]
+  }
+
+  override def toString() = "UnitTypeInfo"
+
+  override def equals(obj: scala.Any) = {
+    obj.isInstanceOf[UnitTypeInfo]
+  }
+
+  override def hashCode() = classOf[UnitTypeInfo].hashCode
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/64915590/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
index 1170129..70276cb 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
@@ -20,13 +20,14 @@ package org.apache.flink.api.scala.types
 import java.io.{DataInput, DataOutput}
 
 import org.apache.flink.api.java.`type`.extractor.TypeExtractorTest.CustomTuple
+import org.apache.flink.api.java.io.CollectionInputFormat
 import org.apache.hadoop.io.Writable
 import org.junit.{Assert, Test}
 
 import org.apache.flink.api.common.typeinfo._
 import org.apache.flink.api.java.typeutils._
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.api.scala.typeutils.{UnitTypeInfo, CaseClassTypeInfo}
 import org.apache.flink.types.{IntValue, StringValue}
 
 class MyWritable extends Writable {
@@ -592,5 +593,16 @@ class TypeInformationGenTest {
     }
     // (Do not call g, because it throws NotImplementedError. This is a compile time test.)
   }
+
+  @Test
+  def testUnit(): Unit = {
+    val ti = createTypeInformation[Unit]
+    Assert.assertTrue(ti.isInstanceOf[UnitTypeInfo])
+
+    // This checks the condition in checkCollection. If this fails with IllegalArgumentException,
+    // then things like "env.fromElements((),(),())" won't work.
+    import scala.collection.JavaConversions._
+    CollectionInputFormat.checkCollection(Seq((),(),()), (new UnitTypeInfo).getTypeClass)
+  }
 }
 


[02/10] flink git commit: [FLINK-2725] Add Max/Min/Sum aggregation for mutable types.

Posted by fh...@apache.org.
[FLINK-2725] Add Max/Min/Sum aggregation for mutable types.

This closes #1191


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

Branch: refs/heads/master
Commit: da248b15e1b1dbe09345d3bb186dc815a45e9a3c
Parents: 6491559
Author: Greg Hogan <co...@greghogan.com>
Authored: Tue Sep 22 13:01:47 2015 -0400
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 19 15:39:28 2015 +0200

----------------------------------------------------------------------
 .../aggregation/MaxAggregationFunction.java     |  83 ++-
 .../aggregation/MinAggregationFunction.java     |  85 ++-
 .../aggregation/SumAggregationFunction.java     | 190 ++++-
 .../flink/api/java/typeutils/ValueTypeInfo.java |  13 +-
 .../test/javaApiOperators/AggregateITCase.java  |  71 ++
 .../util/ValueCollectionDataSets.java           | 730 +++++++++++++++++++
 6 files changed, 1110 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MaxAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MaxAggregationFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MaxAggregationFunction.java
index f25ca87..59d9e13 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MaxAggregationFunction.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MaxAggregationFunction.java
@@ -18,35 +18,74 @@
 
 package org.apache.flink.api.java.aggregation;
 
+import org.apache.flink.types.CopyableValue;
+import org.apache.flink.types.ResettableValue;
 
-public class MaxAggregationFunction<T extends Comparable<T>> extends AggregationFunction<T> {
-	private static final long serialVersionUID = 1L;
 
-	private T value;
+public abstract class MaxAggregationFunction<T extends Comparable<T>> extends AggregationFunction<T> {
+	private static final long serialVersionUID = 1L;
 
 	@Override
-	public void initializeAggregate() {
-		value = null;
+	public String toString() {
+		return "MAX";
 	}
 
-	@Override
-	public void aggregate(T val) {
-		if (value != null) {
-			int cmp = value.compareTo(val);
-			value = (cmp > 0) ? value : val;
-		} else {
-			value = val;
+	// --------------------------------------------------------------------------------------------
+
+	public static final class ImmutableMaxAgg<U extends Comparable<U>> extends MaxAggregationFunction<U> {
+		private static final long serialVersionUID = 1L;
+
+		private U value;
+
+		@Override
+		public void initializeAggregate() {
+			value = null;
 		}
-	}
 
-	@Override
-	public T getAggregate() {
-		return value;
+		@Override
+		public void aggregate(U val) {
+			if (value != null) {
+				int cmp = value.compareTo(val);
+				value = (cmp > 0) ? value : val;
+			} else {
+				value = val;
+			}
+		}
+
+		@Override
+		public U getAggregate() {
+			return value;
+		}
 	}
 	
-	@Override
-	public String toString() {
-		return "MAX";
+	// --------------------------------------------------------------------------------------------
+
+	public static final class MutableMaxAgg<U extends Comparable<U> & ResettableValue<U> & CopyableValue<U>> extends MaxAggregationFunction<U> {
+		private static final long serialVersionUID = 1L;
+
+		private U value;
+
+		@Override
+		public void initializeAggregate() {
+			value = null;
+		}
+
+		@Override
+		public void aggregate(U val) {
+			if (value != null) {
+				int cmp = value.compareTo(val);
+				if (cmp < 0) {
+					value.setValue(val);
+				}
+			} else {
+				value = val.copy();
+			}
+		}
+
+		@Override
+		public U getAggregate() {
+			return value;
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -58,7 +97,11 @@ public class MaxAggregationFunction<T extends Comparable<T>> extends Aggregation
 		@Override
 		public <T> AggregationFunction<T> createAggregationFunction(Class<T> type) {
 			if (Comparable.class.isAssignableFrom(type)) {
-				return (AggregationFunction<T>) new MaxAggregationFunction();
+				if (ResettableValue.class.isAssignableFrom(type) & CopyableValue.class.isAssignableFrom(type)) {
+					return (AggregationFunction<T>) new MutableMaxAgg();
+				} else {
+					return (AggregationFunction<T>) new ImmutableMaxAgg();
+				}
 			} else {
 				throw new UnsupportedAggregationTypeException("The type " + type.getName() + 
 					" is not supported for maximum aggregation. " +

http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MinAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MinAggregationFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MinAggregationFunction.java
index faf28a7..b72b0f4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MinAggregationFunction.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/MinAggregationFunction.java
@@ -18,35 +18,74 @@
 
 package org.apache.flink.api.java.aggregation;
 
+import org.apache.flink.types.CopyableValue;
+import org.apache.flink.types.ResettableValue;
 
-public class MinAggregationFunction<T extends Comparable<T>> extends AggregationFunction<T> {
-	private static final long serialVersionUID = 1L;
 
-	private T value;
+public abstract class MinAggregationFunction<T extends Comparable<T>> extends AggregationFunction<T> {
+	private static final long serialVersionUID = 1L;
 
 	@Override
-	public void initializeAggregate() {
-		value = null;
+	public String toString() {
+		return "MIN";
 	}
 
-	@Override
-	public void aggregate(T val) {
-		if (value != null) {
-			int cmp = value.compareTo(val);
-			value = (cmp < 0) ? value : val;
-		} else {
-			value = val;
+	// --------------------------------------------------------------------------------------------
+
+	public static final class ImmutableMinAgg<U extends Comparable<U>> extends MinAggregationFunction<U> {
+		private static final long serialVersionUID = 1L;
+
+		private U value;
+
+		@Override
+		public void initializeAggregate() {
+			value = null;
 		}
-	}
 
-	@Override
-	public T getAggregate() {
-		return value;
+		@Override
+		public void aggregate(U val) {
+			if (value != null) {
+				int cmp = value.compareTo(val);
+				value = (cmp < 0) ? value : val;
+			} else {
+				value = val;
+			}
+		}
+
+		@Override
+		public U getAggregate() {
+			return value;
+		}
 	}
-	
-	@Override
-	public String toString() {
-		return "MIN";
+
+	// --------------------------------------------------------------------------------------------
+
+	public static final class MutableMinAgg<U extends Comparable<U> & ResettableValue<U> & CopyableValue<U>> extends MinAggregationFunction<U> {
+		private static final long serialVersionUID = 1L;
+
+		private U value;
+
+		@Override
+		public void initializeAggregate() {
+			value = null;
+		}
+
+		@Override
+		public void aggregate(U val) {
+			if (value != null) {
+				int cmp = value.compareTo(val);
+				if (cmp > 0) {
+					value.setValue(val);
+				}
+			} else {
+				value = val.copy();
+			}
+		}
+
+		@Override
+		public U getAggregate() {
+			return value;
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -58,7 +97,11 @@ public class MinAggregationFunction<T extends Comparable<T>> extends Aggregation
 		@Override
 		public <T> AggregationFunction<T> createAggregationFunction(Class<T> type) {
 			if (Comparable.class.isAssignableFrom(type)) {
-				return (AggregationFunction<T>) new MinAggregationFunction();
+				if (ResettableValue.class.isAssignableFrom(type) & CopyableValue.class.isAssignableFrom(type)) {
+					return (AggregationFunction<T>) new MutableMinAgg();
+				} else {
+					return (AggregationFunction<T>) new ImmutableMinAgg();
+				}
 			} else {
 				throw new UnsupportedAggregationTypeException("The type " + type.getName() + 
 					" is not supported for minimum aggregation. " +

http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-java/src/main/java/org/apache/flink/api/java/aggregation/SumAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/SumAggregationFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/SumAggregationFunction.java
index 24e8f31..ad4644b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/aggregation/SumAggregationFunction.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/aggregation/SumAggregationFunction.java
@@ -18,21 +18,27 @@
 
 package org.apache.flink.api.java.aggregation;
 
+import org.apache.flink.types.ByteValue;
+import org.apache.flink.types.DoubleValue;
+import org.apache.flink.types.FloatValue;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.ShortValue;
 
 public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
-	
+
 	private static final long serialVersionUID = 1L;
 
 	@Override
 	public String toString() {
 		return "SUM";
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static final class ByteSumAgg extends SumAggregationFunction<Byte> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private long agg;
 
 		@Override
@@ -50,10 +56,31 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 			return (byte) agg;
 		}
 	}
-	
+
+	public static final class ByteValueSumAgg extends SumAggregationFunction<ByteValue> {
+		private static final long serialVersionUID = 1L;
+
+		private long agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0;
+		}
+
+		@Override
+		public void aggregate(ByteValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public ByteValue getAggregate() {
+			return new ByteValue((byte) agg);
+		}
+	}
+
 	public static final class ShortSumAgg extends SumAggregationFunction<Short> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private long agg;
 
 		@Override
@@ -71,10 +98,31 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 			return (short) agg;
 		}
 	}
-	
+
+	public static final class ShortValueSumAgg extends SumAggregationFunction<ShortValue> {
+		private static final long serialVersionUID = 1L;
+
+		private long agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0;
+		}
+
+		@Override
+		public void aggregate(ShortValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public ShortValue getAggregate() {
+			return new ShortValue((short) agg);
+		}
+	}
+
 	public static final class IntSumAgg extends SumAggregationFunction<Integer> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private long agg;
 
 		@Override
@@ -92,10 +140,31 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 			return (int) agg;
 		}
 	}
-	
+
+	public static final class IntValueSumAgg extends SumAggregationFunction<IntValue> {
+		private static final long serialVersionUID = 1L;
+
+		private long agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0;
+		}
+
+		@Override
+		public void aggregate(IntValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public IntValue getAggregate() {
+			return new IntValue((int) agg);
+		}
+	}
+
 	public static final class LongSumAgg extends SumAggregationFunction<Long> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private long agg;
 
 		@Override
@@ -113,11 +182,32 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 			return agg;
 		}
 	}
-	
+
+	public static final class LongValueSumAgg extends SumAggregationFunction<LongValue> {
+		private static final long serialVersionUID = 1L;
+
+		private long agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0L;
+		}
+
+		@Override
+		public void aggregate(LongValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public LongValue getAggregate() {
+			return new LongValue(agg);
+		}
+	}
+
 	public static final class FloatSumAgg extends SumAggregationFunction<Float> {
 		private static final long serialVersionUID = 1L;
-		
-		private float agg;
+
+		private double agg;
 
 		@Override
 		public void initializeAggregate() {
@@ -131,13 +221,34 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 
 		@Override
 		public Float getAggregate() {
-			return agg;
+			return (float) agg;
+		}
+	}
+
+	public static final class FloatValueSumAgg extends SumAggregationFunction<FloatValue> {
+		private static final long serialVersionUID = 1L;
+
+		private double agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0.0f;
+		}
+
+		@Override
+		public void aggregate(FloatValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public FloatValue getAggregate() {
+			return new FloatValue((float) agg);
 		}
 	}
-	
+
 	public static final class DoubleSumAgg extends SumAggregationFunction<Double> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private double agg;
 
 		@Override
@@ -155,36 +266,75 @@ public abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
 			return agg;
 		}
 	}
-	
+
+	public static final class DoubleValueSumAgg extends SumAggregationFunction<DoubleValue> {
+		private static final long serialVersionUID = 1L;
+
+		private double agg;
+
+		@Override
+		public void initializeAggregate() {
+			agg = 0.0;
+		}
+
+		@Override
+		public void aggregate(DoubleValue value) {
+			agg += value.getValue();
+		}
+
+		@Override
+		public DoubleValue getAggregate() {
+			return new DoubleValue(agg);
+		}
+	}
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static final class SumAggregationFunctionFactory implements AggregationFunctionFactory {
 		private static final long serialVersionUID = 1L;
-		
+
 		@SuppressWarnings("unchecked")
 		@Override
 		public <T> AggregationFunction<T> createAggregationFunction(Class<T> type) {
 			if (type == Long.class) {
 				return (AggregationFunction<T>) new LongSumAgg();
 			}
+			else if (type == LongValue.class) {
+				return (AggregationFunction<T>) new LongValueSumAgg();
+			}
 			else if (type == Integer.class) {
 				return (AggregationFunction<T>) new IntSumAgg();
 			}
+			else if (type == IntValue.class) {
+				return (AggregationFunction<T>) new IntValueSumAgg();
+			}
 			else if (type == Double.class) {
 				return (AggregationFunction<T>) new DoubleSumAgg();
 			}
+			else if (type == DoubleValue.class) {
+				return (AggregationFunction<T>) new DoubleValueSumAgg();
+			}
 			else if (type == Float.class) {
 				return (AggregationFunction<T>) new FloatSumAgg();
 			}
+			else if (type == FloatValue.class) {
+				return (AggregationFunction<T>) new FloatValueSumAgg();
+			}
 			else if (type == Byte.class) {
 				return (AggregationFunction<T>) new ByteSumAgg();
 			}
+			else if (type == ByteValue.class) {
+				return (AggregationFunction<T>) new ByteValueSumAgg();
+			}
 			else if (type == Short.class) {
 				return (AggregationFunction<T>) new ShortSumAgg();
 			}
+			else if (type == ShortValue.class) {
+				return (AggregationFunction<T>) new ShortValueSumAgg();
+			}
 			else {
 				throw new UnsupportedAggregationTypeException("The type " + type.getName() + 
-					" has currently not supported for built-in sum aggregations.");
+					" is currently not supported for built-in sum aggregations.");
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
index 0b4823e..5187de7 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
@@ -51,7 +51,18 @@ import org.apache.flink.types.Value;
 public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implements AtomicType<T> {
 
 	private static final long serialVersionUID = 1L;
-	
+
+	public static final ValueTypeInfo<BooleanValue> BOOLEAN_VALUE_TYPE_INFO = new ValueTypeInfo<>(BooleanValue.class);
+	public static final ValueTypeInfo<ByteValue> BYTE_VALUE_TYPE_INFO = new ValueTypeInfo<>(ByteValue.class);
+	public static final ValueTypeInfo<CharValue> CHAR_VALUE_TYPE_INFO = new ValueTypeInfo<>(CharValue.class);
+	public static final ValueTypeInfo<DoubleValue> DOUBLE_VALUE_TYPE_INFO = new ValueTypeInfo<>(DoubleValue.class);
+	public static final ValueTypeInfo<FloatValue> FLOAT_VALUE_TYPE_INFO = new ValueTypeInfo<>(FloatValue.class);
+	public static final ValueTypeInfo<IntValue> INT_VALUE_TYPE_INFO = new ValueTypeInfo<>(IntValue.class);
+	public static final ValueTypeInfo<LongValue> LONG_VALUE_TYPE_INFO = new ValueTypeInfo<>(LongValue.class);
+	public static final ValueTypeInfo<NullValue> NULL_VALUE_TYPE_INFO = new ValueTypeInfo<>(NullValue.class);
+	public static final ValueTypeInfo<ShortValue> SHORT_VALUE_TYPE_INFO = new ValueTypeInfo<>(ShortValue.class);
+	public static final ValueTypeInfo<StringValue> STRING_VALUE_TYPE_INFO = new ValueTypeInfo<>(StringValue.class);
+
 	private final Class<T> type;
 	
 	public ValueTypeInfo(Class<T> type) {

http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
index d02f228..fc01ce7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
@@ -25,7 +25,11 @@ import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.javaApiOperators.util.ValueCollectionDataSets;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.StringValue;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -62,6 +66,27 @@ public class AggregateITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
+	public void testFullAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Full Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds
+				.aggregate(Aggregations.SUM, 0)
+				.and(Aggregations.MAX, 1)
+				.project(0, 1);
+
+		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
+
+		String expected = "231,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
 	public void testGroupedAggregate() throws Exception {
 		/*
 		 * Grouped Aggregate
@@ -87,6 +112,31 @@ public class AggregateITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
+	public void testGroupedAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Grouped Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.SUM, 0)
+				.project(1, 0);
+
+		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
+
+		String expected = "1,1\n" +
+				"2,5\n" +
+				"3,15\n" +
+				"4,34\n" +
+				"5,65\n" +
+				"6,111\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
 	public void testNestedAggregate() throws Exception {
 		/*
 		 * Nested Aggregate
@@ -106,4 +156,25 @@ public class AggregateITCase extends MultipleProgramsTestBase {
 
 		compareResultAsTuples(result, expected);
 	}
+
+	@Test
+	public void testNestedAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Nested Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple1<IntValue>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.MIN, 0)
+				.aggregate(Aggregations.MIN, 0)
+				.project(0);
+
+		List<Tuple1<IntValue>> result = aggregateDs.collect();
+
+		String expected = "1\n";
+
+		compareResultAsTuples(result, expected);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/da248b15/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
new file mode 100644
index 0000000..04a7bc5
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
@@ -0,0 +1,730 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.ValueTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.StringValue;
+import org.apache.hadoop.io.IntWritable;
+
+import scala.math.BigInt;
+
+/**
+ * #######################################################################################################
+ * 
+ * 			BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. 
+ * 			IF YOU MODIFY THE DATA MAKE SURE YOU CHECK THAT ALL TESTS ARE STILL WORKING!
+ * 
+ * #######################################################################################################
+ */
+public class ValueCollectionDataSets {
+
+	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> get3TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new IntValue(1), new LongValue(1l), new StringValue("Hi")));
+		data.add(new Tuple3<>(new IntValue(2), new LongValue(2l), new StringValue("Hello")));
+		data.add(new Tuple3<>(new IntValue(3), new LongValue(2l), new StringValue("Hello world")));
+		data.add(new Tuple3<>(new IntValue(4), new LongValue(3l), new StringValue("Hello world, how are you?")));
+		data.add(new Tuple3<>(new IntValue(5), new LongValue(3l), new StringValue("I am fine.")));
+		data.add(new Tuple3<>(new IntValue(6), new LongValue(3l), new StringValue("Luke Skywalker")));
+		data.add(new Tuple3<>(new IntValue(7), new LongValue(4l), new StringValue("Comment#1")));
+		data.add(new Tuple3<>(new IntValue(8), new LongValue(4l), new StringValue("Comment#2")));
+		data.add(new Tuple3<>(new IntValue(9), new LongValue(4l), new StringValue("Comment#3")));
+		data.add(new Tuple3<>(new IntValue(10), new LongValue(4l), new StringValue("Comment#4")));
+		data.add(new Tuple3<>(new IntValue(11), new LongValue(5l), new StringValue("Comment#5")));
+		data.add(new Tuple3<>(new IntValue(12), new LongValue(5l), new StringValue("Comment#6")));
+		data.add(new Tuple3<>(new IntValue(13), new LongValue(5l), new StringValue("Comment#7")));
+		data.add(new Tuple3<>(new IntValue(14), new LongValue(5l), new StringValue("Comment#8")));
+		data.add(new Tuple3<>(new IntValue(15), new LongValue(5l), new StringValue("Comment#9")));
+		data.add(new Tuple3<>(new IntValue(16), new LongValue(6l), new StringValue("Comment#10")));
+		data.add(new Tuple3<>(new IntValue(17), new LongValue(6l), new StringValue("Comment#11")));
+		data.add(new Tuple3<>(new IntValue(18), new LongValue(6l), new StringValue("Comment#12")));
+		data.add(new Tuple3<>(new IntValue(19), new LongValue(6l), new StringValue("Comment#13")));
+		data.add(new Tuple3<>(new IntValue(20), new LongValue(6l), new StringValue("Comment#14")));
+		data.add(new Tuple3<>(new IntValue(21), new LongValue(6l), new StringValue("Comment#15")));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> getSmall3TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new IntValue(1), new LongValue(1l), new StringValue("Hi")));
+		data.add(new Tuple3<>(new IntValue(2), new LongValue(2l), new StringValue("Hello")));
+		data.add(new Tuple3<>(new IntValue(3), new LongValue(2l), new StringValue("Hello world")));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> get5TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
+
+		data.add(new Tuple5<>(new IntValue(1), new LongValue(1l), new IntValue(0), new StringValue("Hallo"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(2l), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(3l), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(4l), new IntValue(3), new StringValue("Hallo Welt wie gehts?"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(5l), new IntValue(4), new StringValue("ABC"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(6l), new IntValue(5), new StringValue("BCD"), new LongValue(3l)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(7l), new IntValue(6), new StringValue("CDE"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(8l), new IntValue(7), new StringValue("DEF"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(9l), new IntValue(8), new StringValue("EFG"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(10l), new IntValue(9), new StringValue("FGH"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(11l), new IntValue(10), new StringValue("GHI"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(12l), new IntValue(11), new StringValue("HIJ"), new LongValue(3l)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(13l), new IntValue(12), new StringValue("IJK"), new LongValue(3l)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(14l), new IntValue(13), new StringValue("JKL"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(15l), new IntValue(14), new StringValue("KLM"), new LongValue(2l)));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
+			TupleTypeInfo<>(
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> getSmall5TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
+
+		data.add(new Tuple5<>(new IntValue(1), new LongValue(1l), new IntValue(0), new StringValue("Hallo"), new LongValue(1l)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(2l), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2l)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(3l), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1l)));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
+			TupleTypeInfo<>(
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getSmallNestedTupleDataSet(ExecutionEnvironment env) {
+		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(1)), new StringValue("one")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("two")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("three")));
+
+		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) {
+		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(3)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(2)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(1)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("b")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("c")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(6)), new StringValue("c")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(4), new IntValue(9)), new StringValue("c")));
+
+		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) {
+		List<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(3)), new StringValue("a"), new IntValue(2)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(2)), new StringValue("a"), new IntValue(1)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(1)), new StringValue("a"), new IntValue(3)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(2)), new StringValue("b"), new IntValue(4)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(3)), new StringValue("c"), new IntValue(5)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(6)), new StringValue("c"), new IntValue(6)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(4), new IntValue(9)), new StringValue("c"), new IntValue(7)));
+
+		TupleTypeInfo<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+	
+	public static DataSet<StringValue> getStringDataSet(ExecutionEnvironment env) {
+		List<StringValue> data = new ArrayList<>();
+
+		data.add(new StringValue("Hi"));
+		data.add(new StringValue("Hello"));
+		data.add(new StringValue("Hello world"));
+		data.add(new StringValue("Hello world, how are you?"));
+		data.add(new StringValue("I am fine."));
+		data.add(new StringValue("Luke Skywalker"));
+		data.add(new StringValue("Random comment"));
+		data.add(new StringValue("LOL"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<IntValue> getIntDataSet(ExecutionEnvironment env) {
+		List<IntValue> data = new ArrayList<>();
+
+		data.add(new IntValue(1));
+		data.add(new IntValue(2));
+		data.add(new IntValue(2));
+		data.add(new IntValue(3));
+		data.add(new IntValue(3));
+		data.add(new IntValue(3));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<CustomType> getCustomTypeDataSet(ExecutionEnvironment env) {
+		List<CustomType> data = new ArrayList<CustomType>();
+
+		data.add(new CustomType(1, 0l, "Hi"));
+		data.add(new CustomType(2, 1l, "Hello"));
+		data.add(new CustomType(2, 2l, "Hello world"));
+		data.add(new CustomType(3, 3l, "Hello world, how are you?"));
+		data.add(new CustomType(3, 4l, "I am fine."));
+		data.add(new CustomType(3, 5l, "Luke Skywalker"));
+		data.add(new CustomType(4, 6l, "Comment#1"));
+		data.add(new CustomType(4, 7l, "Comment#2"));
+		data.add(new CustomType(4, 8l, "Comment#3"));
+		data.add(new CustomType(4, 9l, "Comment#4"));
+		data.add(new CustomType(5, 10l, "Comment#5"));
+		data.add(new CustomType(5, 11l, "Comment#6"));
+		data.add(new CustomType(5, 12l, "Comment#7"));
+		data.add(new CustomType(5, 13l, "Comment#8"));
+		data.add(new CustomType(5, 14l, "Comment#9"));
+		data.add(new CustomType(6, 15l, "Comment#10"));
+		data.add(new CustomType(6, 16l, "Comment#11"));
+		data.add(new CustomType(6, 17l, "Comment#12"));
+		data.add(new CustomType(6, 18l, "Comment#13"));
+		data.add(new CustomType(6, 19l, "Comment#14"));
+		data.add(new CustomType(6, 20l, "Comment#15"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<CustomType> getSmallCustomTypeDataSet(ExecutionEnvironment env) {
+		List<CustomType> data = new ArrayList<CustomType>();
+
+		data.add(new CustomType(1, 0l, "Hi"));
+		data.add(new CustomType(2, 1l, "Hello"));
+		data.add(new CustomType(2, 2l, "Hello world"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static class CustomType implements Serializable {
+
+		private static final long serialVersionUID = 1L;
+
+		public IntValue myInt;
+		public LongValue myLong;
+		public StringValue myString;
+
+		public CustomType() {
+		}
+
+		public CustomType(int i, long l, String s) {
+			myInt = new IntValue(i);
+			myLong = new LongValue(l);
+			myString = new StringValue(s);
+		}
+
+		@Override
+		public String toString() {
+			return myInt + "," + myLong + "," + myString;
+		}
+	}
+
+	public static class CustomTypeComparator implements Comparator<CustomType> {
+
+		@Override
+		public int compare(CustomType o1, CustomType o2) {
+			int diff = o1.myInt.getValue() - o2.myInt.getValue();
+			if (diff != 0) {
+				return diff;
+			}
+			diff = (int) (o1.myLong.getValue() - o2.myLong.getValue());
+			return diff != 0 ? diff : o1.myString.getValue().compareTo(o2.myString.getValue());
+		}
+
+	}
+
+	public static DataSet<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> getSmallTuplebasedDataSet(ExecutionEnvironment env) {
+		List<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> data = new ArrayList<>();
+		
+		data.add(new Tuple7<>(new IntValue(1), new StringValue("First"), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new LongValue(10000L)));
+		data.add(new Tuple7<>(new IntValue(2), new StringValue("Second"), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new LongValue(20000L)));
+		data.add(new Tuple7<>(new IntValue(3), new StringValue("Third"), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new LongValue(30000L)));
+
+		return env.fromCollection(data);
+	}
+	
+	public static DataSet<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) {
+		List<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> data = new ArrayList<>();
+		
+		data.add(new Tuple7<>(new LongValue(10000L), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new IntValue(1), new StringValue("First")));
+		data.add(new Tuple7<>(new LongValue(20000L), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new IntValue(2), new StringValue("Second")));
+		data.add(new Tuple7<>(new LongValue(30000L), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new IntValue(3), new StringValue("Third")));
+		
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getSmallPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1 /*number*/, "First" /*str*/, 10 /*f0*/, 100/*f1.myInt*/, 1000L/*f1.myLong*/, "One" /*f1.myString*/, 10000L /*nestedPojo.longNumber*/));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getDuplicatePojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getMixedPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x
+		data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L));
+		data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L));
+		data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L));
+		data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L));
+		data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L));
+		data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x
+		data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L));
+
+		return env.fromCollection(data);
+	}
+
+	public static class POJO {
+		public IntValue number;
+		public StringValue str;
+		public Tuple2<IntValue, CustomType> nestedTupleWithCustom;
+		public NestedPojo nestedPojo;
+		public transient LongValue ignoreMe;
+
+		public POJO(int i0, String s0,
+					int i1, int i2, long l0, String s1,
+					long l1) {
+			this.number = new IntValue(i0);
+			this.str = new StringValue(s0);
+			this.nestedTupleWithCustom = new Tuple2<>(new IntValue(i1), new CustomType(i2, l0, s1));
+			this.nestedPojo = new NestedPojo();
+			this.nestedPojo.longNumber = new LongValue(l1);
+		}
+
+		public POJO() {
+		}
+
+		@Override
+		public String toString() {
+			return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber;
+		}
+	}
+
+	public static class NestedPojo {
+		public static Object ignoreMe;
+		public LongValue longNumber;
+
+		public NestedPojo() {
+		}
+	}
+
+	public static DataSet<CrazyNested> getCrazyNestedDataSet(ExecutionEnvironment env) {
+		List<CrazyNested> data = new ArrayList<CrazyNested>();
+
+		data.add(new CrazyNested("aa"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+
+		return env.fromCollection(data);
+	}
+
+	public static class CrazyNested {
+		public CrazyNestedL1 nest_Lvl1;
+		public LongValue something; // test proper null-value handling
+
+		public CrazyNested() {
+		}
+
+		public CrazyNested(String set, String second, long s) { // additional CTor to set all fields to non-null values
+			this(set);
+			something = new LongValue(s);
+			nest_Lvl1.a = new StringValue(second);
+		}
+
+		public CrazyNested(String set) {
+			nest_Lvl1 = new CrazyNestedL1();
+			nest_Lvl1.nest_Lvl2 = new CrazyNestedL2();
+			nest_Lvl1.nest_Lvl2.nest_Lvl3 = new CrazyNestedL3();
+			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4 = new CrazyNestedL4();
+			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal = new StringValue(set);
+		}
+	}
+
+	public static class CrazyNestedL1 {
+		public StringValue a;
+		public IntValue b;
+		public CrazyNestedL2 nest_Lvl2;
+	}
+
+	public static class CrazyNestedL2 {
+		public CrazyNestedL3 nest_Lvl3;
+	}
+
+	public static class CrazyNestedL3 {
+		public CrazyNestedL4 nest_Lvl4;
+	}
+
+	public static class CrazyNestedL4 {
+		public StringValue f1nal;
+	}
+
+	// Copied from TypeExtractorTest
+	public static class FromTuple extends Tuple3<StringValue, StringValue, LongValue> {
+		private static final long serialVersionUID = 1L;
+		public IntValue special;
+	}
+
+	public static class FromTupleWithCTor extends FromTuple {
+
+		private static final long serialVersionUID = 1L;
+
+		public FromTupleWithCTor() {}
+
+		public FromTupleWithCTor(int special, long tupleField) {
+			this.special = new IntValue(special);
+			this.setField(new LongValue(tupleField), 2);
+		}
+	}
+
+	public static DataSet<FromTupleWithCTor> getPojoExtendingFromTuple(ExecutionEnvironment env) {
+		List<FromTupleWithCTor> data = new ArrayList<>();
+		data.add(new FromTupleWithCTor(1, 10L)); // 3x
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(2, 20L)); // 2x
+		data.add(new FromTupleWithCTor(2, 20L));
+		return env.fromCollection(data);
+	}
+
+	public static class PojoContainingTupleAndWritable {
+		public IntValue someInt;
+		public StringValue someString;
+		public IntWritable hadoopFan;
+		public Tuple2<LongValue, LongValue> theTuple;
+
+		public PojoContainingTupleAndWritable() {
+		}
+
+		public PojoContainingTupleAndWritable(int i, long l1, long l2) {
+			hadoopFan = new IntWritable(i);
+			someInt = new IntValue(i);
+			theTuple = new Tuple2<>(new LongValue(l1), new LongValue(l2));
+		}
+	}
+
+	public static DataSet<PojoContainingTupleAndWritable> getPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		return env.fromCollection(data);
+	}
+
+
+
+	public static DataSet<PojoContainingTupleAndWritable> getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 201L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 600L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 400L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<IntValue, CrazyNested, POJO>> getTupleContainingPojos(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, CrazyNested, POJO>> data = new ArrayList<>();
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		// POJO is not initialized according to the first two fields.
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(2), new CrazyNested("two", "duo", 2L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x
+		return env.fromCollection(data);
+	}
+
+	public static class Pojo1 {
+		public StringValue a;
+		public StringValue b;
+
+		public Pojo1() {}
+
+		public Pojo1(String a, String b) {
+			this.a = new StringValue(a);
+			this.b = new StringValue(b);
+		}
+	}
+
+	public static class Pojo2 {
+		public StringValue a2;
+		public StringValue b2;
+	}
+
+	public static class PojoWithMultiplePojos {
+		public Pojo1 p1;
+		public Pojo2 p2;
+		public IntValue i0;
+
+		public PojoWithMultiplePojos() {
+		}
+
+		public PojoWithMultiplePojos(String a, String b, String a1, String b1, int i0) {
+			p1 = new Pojo1();
+			p1.a = new StringValue(a);
+			p1.b = new StringValue(b);
+			p2 = new Pojo2();
+			p2.a2 = new StringValue(a1);
+			p2.b2 = new StringValue(b1);
+			this.i0 = new IntValue(i0);
+		}
+	}
+
+	public static DataSet<PojoWithMultiplePojos> getPojoWithMultiplePojos(ExecutionEnvironment env) {
+		List<PojoWithMultiplePojos> data = new ArrayList<>();
+		data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		return env.fromCollection(data);
+	}
+
+	public enum Category {
+		CAT_A, CAT_B;
+	}
+
+	public static class PojoWithDateAndEnum {
+		public StringValue group;
+		public Date date;
+		public Category cat;
+	}
+	
+	public static DataSet<PojoWithDateAndEnum> getPojoWithDateAndEnum(ExecutionEnvironment env) {
+		List<PojoWithDateAndEnum> data = new ArrayList<PojoWithDateAndEnum>();
+		
+		PojoWithDateAndEnum one = new PojoWithDateAndEnum();
+		one.group = new StringValue("a");
+		one.date = new Date(666);
+		one.cat = Category.CAT_A;
+		data.add(one);
+		
+		PojoWithDateAndEnum two = new PojoWithDateAndEnum();
+		two.group = new StringValue("a");
+		two.date = new Date(666);
+		two.cat = Category.CAT_A;
+		data.add(two);
+		
+		PojoWithDateAndEnum three = new PojoWithDateAndEnum();
+		three.group = new StringValue("b");
+		three.date = new Date(666);
+		three.cat = Category.CAT_B;
+		data.add(three);
+		
+		return env.fromCollection(data);
+	}
+
+	public static class PojoWithCollection {
+		public List<Pojo1> pojos;
+		public IntValue key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	public static class PojoWithCollectionGeneric {
+		public List<Pojo1> pojos;
+		public IntValue key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+		private PojoWithDateAndEnum makeMeGeneric;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	public static DataSet<PojoWithCollection> getPojoWithCollection(ExecutionEnvironment env) {
+		List<PojoWithCollection> data = new ArrayList<>();
+
+		List<Pojo1> pojosList1 = new ArrayList<>();
+		pojosList1.add(new Pojo1("a", "aa"));
+		pojosList1.add(new Pojo1("b", "bb"));
+
+		List<Pojo1> pojosList2 = new ArrayList<>();
+		pojosList2.add(new Pojo1("a2", "aa2"));
+		pojosList2.add(new Pojo1("b2", "bb2"));
+
+		PojoWithCollection pwc1 = new PojoWithCollection();
+		pwc1.pojos = pojosList1;
+		pwc1.key = new IntValue(0);
+		pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc1.scalaBigInt = BigInt.int2bigInt(10);
+		pwc1.bigDecimalKeepItNull = null;
+		
+		// use calendar to make it stable across time zones
+		GregorianCalendar gcl1 = new GregorianCalendar(2033, 04, 18);
+		pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis());
+		pwc1.mixed = new ArrayList<Object>();
+		Map<StringValue, IntValue> map = new HashMap<>();
+		map.put(new StringValue("someKey"), new IntValue(1));
+		pwc1.mixed.add(map);
+		pwc1.mixed.add(new File("/this/is/wrong"));
+		pwc1.mixed.add("uhlala");
+
+		PojoWithCollection pwc2 = new PojoWithCollection();
+		pwc2.pojos = pojosList2;
+		pwc2.key = new IntValue(0);
+		pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc2.scalaBigInt = BigInt.int2bigInt(31104000);
+		pwc2.bigDecimalKeepItNull = null;
+		
+		GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3);
+		pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976
+
+		data.add(pwc1);
+		data.add(pwc2);
+
+		return env.fromCollection(data);
+	}
+
+}
+


[06/10] flink git commit: [FLINK-2107] Add hash-based strategies for left and right outer joins.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
index 2a306ca..582077e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
@@ -42,8 +42,8 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.TupleMatch;
-import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.TupleMatchRemovingJoin;
+import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatchRemovingJoin;
+import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatch;
 import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
@@ -59,6 +59,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.joinTuples;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.collectTupleData;
+
 /**
  * Test specialized hash join that keeps the build side data (in memory and on hard disk)
  * This is used for iterative tasks.
@@ -203,7 +206,7 @@ public class ReusingReOpenableHashTableITCase {
 	
 	private void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TestData.TupleGenerator bgen, TestData.TupleGenerator pgen) throws Exception {
 		// collect expected data
-		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = ReusingHashMatchIteratorITCase.matchSecondTupleFields(ReusingHashMatchIteratorITCase.collectTupleData(buildInput), ReusingHashMatchIteratorITCase.collectTupleData(probeInput));
+		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = joinTuples(collectTupleData(buildInput), collectTupleData(probeInput));
 		
 		final List<Map<Integer, Collection<TupleMatch>>> expectedNMatchesMapList = new ArrayList<>(NUM_PROBES);
 		final FlatJoinFunction[] nMatcher = new TupleMatchRemovingJoin[NUM_PROBES];
@@ -224,11 +227,11 @@ public class ReusingReOpenableHashTableITCase {
 		probeInput.reset();
 
 		// compare with iterator values
-		ReusingBuildFirstReOpenableHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-				new ReusingBuildFirstReOpenableHashMatchIterator<>(
+		ReusingBuildFirstReOpenableHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+				new ReusingBuildFirstReOpenableHashJoinIterator<>(
 						buildInput, probeInput, this.recordSerializer, this.record1Comparator, 
 					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
 		
 		iterator.open();
 		// do first join with both inputs

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
index 4c5a07e..4f18494 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
@@ -29,8 +29,8 @@ 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.hash.ReusingBuildFirstHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
@@ -180,11 +180,11 @@ public class HashVsSortMiniBenchmark {
 			long start = System.nanoTime();
 			
 			// compare with iterator values
-			final ReusingBuildFirstHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildFirstHashMatchIterator<>(
+			final ReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildFirstHashJoinIterator<>(
 						input1, input2, this.serializer1.getSerializer(), this.comparator1, 
 							this.serializer2.getSerializer(), this.comparator2, this.pairComparator11,
-							this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, true);
+							this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, false, true);
 			
 			iterator.open();
 			
@@ -219,11 +219,11 @@ public class HashVsSortMiniBenchmark {
 			long start = System.nanoTime();
 			
 			// compare with iterator values
-			ReusingBuildSecondHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildSecondHashMatchIterator<>(
+			ReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildSecondHashJoinIterator<>(
 						input1, input2, this.serializer1.getSerializer(), this.comparator1, 
 						this.serializer2.getSerializer(), this.comparator2, this.pairComparator11,
-						this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, true);
+						this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE, false, true);
 			
 			iterator.open();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 dbae59d..7c39085 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
@@ -932,7 +932,13 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    * 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)
+    strategy match {
+      case JoinHint.OPTIMIZER_CHOOSES |
+           JoinHint.REPARTITION_SORT_MERGE =>
+        new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.FULL_OUTER)
+      case _ =>
+        throw new InvalidProgramException("Invalid JoinHint for FullOuterJoin: " + strategy)
+    }
 
   /**
    * An outer join on the left side.
@@ -960,7 +966,15 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    * @see #fullOuterJoin
    */
   def leftOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O] =
-    new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.LEFT_OUTER)
+    strategy match {
+      case JoinHint.OPTIMIZER_CHOOSES |
+           JoinHint.REPARTITION_SORT_MERGE |
+           JoinHint.REPARTITION_HASH_SECOND |
+      JoinHint.BROADCAST_HASH_SECOND =>
+        new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.LEFT_OUTER)
+      case _ =>
+        throw new InvalidProgramException("Invalid JoinHint for LeftOuterJoin: " + strategy)
+    }
 
   /**
    * An outer join on the right side.
@@ -988,7 +1002,15 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
    * @see #fullOuterJoin
    */
   def rightOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O] =
-    new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.RIGHT_OUTER)
+    strategy match {
+      case JoinHint.OPTIMIZER_CHOOSES |
+           JoinHint.REPARTITION_SORT_MERGE |
+           JoinHint.REPARTITION_HASH_FIRST |
+      JoinHint.BROADCAST_HASH_FIRST =>
+        new UnfinishedOuterJoinOperation(this, other, strategy, JoinType.RIGHT_OUTER)
+      case _ =>
+        throw new InvalidProgramException("Invalid JoinHint for RightOuterJoin: " + strategy)
+    }
 
   // --------------------------------------------------------------------------------------------
   //  Co-Group

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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
index ebd1ddf..c2dca66 100644
--- 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
@@ -21,6 +21,7 @@ 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.common.operators.base.JoinOperatorBase.JoinHint;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -50,7 +51,21 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testUDFLeftOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+	public void testLeftOuterJoin1() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testLeftOuterJoin2() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test
+	public void testLeftOuterJoin3() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	private void testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
 		/*
 		 * UDF Join on tuples with key field positions
 		 */
@@ -60,7 +75,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 		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)
+				ds1.leftOuterJoin(ds2, hint)
 						.where(0)
 						.equalTo(0)
 						.with(new T3T5FlatJoin());
@@ -76,7 +91,21 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testUDFRightOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+	public void testRightOuterJoin1() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testRightOuterJoin2() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test
+	public void testRightOuterJoin3() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+	private void testRightOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
 		/*
 		 * UDF Join on tuples with key field positions
 		 */
@@ -86,7 +115,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 		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)
+				ds1.rightOuterJoin(ds2, hint)
 						.where(1)
 						.equalTo(1)
 						.with(new T3T5FlatJoin());
@@ -102,7 +131,11 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testUDFFullOuterJoinOnTuplesWithKeyFieldPositions() throws Exception {
+	public void testFullOuterJoin1() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	private void testFullOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
 		/*
 		 * UDF Join on tuples with key field positions
 		 */
@@ -112,7 +145,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 		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)
+				ds1.fullOuterJoin(ds2, hint)
 						.where(0)
 						.equalTo(2)
 						.with(new T3T5FlatJoin());
@@ -128,7 +161,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception {
+	public void testJoinOnTuplesWithCompositeKeyPositions() throws Exception {
 		/*
 		 * UDF Join on tuples with multiple key field positions
 		 */
@@ -183,7 +216,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector() throws Exception {
+	public void testJoinWithMixedKeyTypes1() throws Exception {
 		/*
 		 * Join on a tuple input with key field selector and a custom type input with key extractor
 		 */
@@ -218,7 +251,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 
 
 	@Test
-	public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
+	public void testJoinWithMixedKeyTypes2()
 			throws Exception {
 		/*
 		 * Join on a tuple input with key field selector and a custom type input with key extractor
@@ -252,7 +285,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception {
+	public void testJoinWithTupleReturningKeySelectors() throws Exception {
 		/*
 		 * UDF Join on tuples with tuple-returning key selectors
 		 */
@@ -296,7 +329,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception {
+	public void testJoinWithNestedKeyExpression1() throws Exception {
 		/*
 		 * Join nested pojo against tuple (selected using a string)
 		 */
@@ -308,7 +341,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 				ds1.fullOuterJoin(ds2)
 						.where("nestedPojo.longNumber")
 						.equalTo("f6")
-						.with(new ProjectBothFunction());
+						.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();
 
@@ -320,7 +353,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exception {
+	public void testJoinWithNestedKeyExpression2() throws Exception {
 		/*
 		 * Join nested pojo against tuple (selected as an integer)
 		 */
@@ -344,7 +377,7 @@ public class OuterJoinITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exception {
+	public void testJoinWithCompositeKeyExpressions() throws Exception {
 		/*
 		 * selecting multiple fields using expression language
 		 */


[08/10] flink git commit: [FLINK-2107] Add hash-based strategies for left and right outer joins.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskExternalITCase.java
index 7c8d04e..f4fa036 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskExternalITCase.java
@@ -37,9 +37,9 @@ import org.junit.Test;
 
 public abstract class AbstractOuterJoinTaskExternalITCase extends BinaryOperatorTestBase<FlatJoinFunction<Tuple2<Integer, Integer>,
 		Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
-	
-	private static final long HASH_MEM = 4 * 1024 * 1024;
-	
+
+	protected static final long HASH_MEM = 4 * 1024 * 1024;
+
 	private static final long SORT_MEM = 3 * 1024 * 1024;
 	
 	private static final long BNLJN_MEM = 10 * PAGE_SIZE;
@@ -47,33 +47,30 @@ public abstract class AbstractOuterJoinTaskExternalITCase extends BinaryOperator
 	private final double bnljn_frac;
 	
 	@SuppressWarnings("unchecked")
-	private final TypeComparator<Tuple2<Integer, Integer>> comparator1 = new TupleComparator<>(
+	protected final TypeComparator<Tuple2<Integer, Integer>> comparator1 = new TupleComparator<>(
 			new int[]{0},
 			new TypeComparator<?>[]{new IntComparator(true)},
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE}
 	);
 	
 	@SuppressWarnings("unchecked")
-	private final TypeComparator<Tuple2<Integer, Integer>> comparator2 = new TupleComparator<>(
+	protected final TypeComparator<Tuple2<Integer, Integer>> comparator2 = new TupleComparator<>(
 			new int[]{0},
 			new TypeComparator<?>[]{new IntComparator(true)},
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE}
 	);
 	
 	@SuppressWarnings("unchecked")
-	private final TypeSerializer<Tuple2<Integer, Integer>> serializer = new TupleSerializer<>(
+	protected final TypeSerializer<Tuple2<Integer, Integer>> serializer = new TupleSerializer<>(
 			(Class<Tuple2<Integer, Integer>>) (Class<?>) Tuple2.class,
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE, IntSerializer.INSTANCE}
 	);
 	
-	private final CountingOutputCollector<Tuple2<Integer, Integer>> output = new CountingOutputCollector<>();
-	
-	private final DriverStrategy driverStrategy;
+	protected final CountingOutputCollector<Tuple2<Integer, Integer>> output = new CountingOutputCollector<>();
 	
-	public AbstractOuterJoinTaskExternalITCase(ExecutionConfig config, DriverStrategy driverStrategy) {
+	public AbstractOuterJoinTaskExternalITCase(ExecutionConfig config) {
 		super(config, HASH_MEM, 2, SORT_MEM);
 		bnljn_frac = (double) BNLJN_MEM / this.getMemoryManager().getMemorySize();
-		this.driverStrategy = driverStrategy;
 	}
 	
 	@Test
@@ -90,7 +87,7 @@ public abstract class AbstractOuterJoinTaskExternalITCase extends BinaryOperator
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortStrategy());
 		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -106,6 +103,8 @@ public abstract class AbstractOuterJoinTaskExternalITCase extends BinaryOperator
 	protected abstract int calculateExpectedCount(int keyCnt1, int valCnt1, int keyCnt2, int valCnt2);
 	
 	protected abstract AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver();
+
+	protected abstract DriverStrategy getSortStrategy();
 	
 	// =================================================================================================
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java
index ad784b5..b265eae 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java
@@ -60,35 +60,32 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 	
 	private final double bnljn_frac;
 	
-	private final DriverStrategy driverStrategy;
-	
 	@SuppressWarnings("unchecked")
-	private final TypeComparator<Tuple2<Integer, Integer>> comparator1 = new TupleComparator<>(
+	protected final TypeComparator<Tuple2<Integer, Integer>> comparator1 = new TupleComparator<>(
 			new int[]{0},
 			new TypeComparator<?>[]{new IntComparator(true)},
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE}
 	);
 	
 	@SuppressWarnings("unchecked")
-	private final TypeComparator<Tuple2<Integer, Integer>> comparator2 = new TupleComparator<>(
+	protected final TypeComparator<Tuple2<Integer, Integer>> comparator2 = new TupleComparator<>(
 			new int[]{0},
 			new TypeComparator<?>[]{new IntComparator(true)},
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE}
 	);
 	
-	private final List<Tuple2<Integer, Integer>> outList = new ArrayList<>();
+	protected final List<Tuple2<Integer, Integer>> outList = new ArrayList<>();
 	
 	@SuppressWarnings("unchecked")
-	private final TypeSerializer<Tuple2<Integer, Integer>> serializer = new TupleSerializer<>(
+	protected final TypeSerializer<Tuple2<Integer, Integer>> serializer = new TupleSerializer<>(
 			(Class<Tuple2<Integer, Integer>>) (Class<?>) Tuple2.class,
 			new TypeSerializer<?>[]{IntSerializer.INSTANCE, IntSerializer.INSTANCE}
 	);
 	
 	
-	public AbstractOuterJoinTaskTest(ExecutionConfig config, DriverStrategy driverStrategy) {
+	public AbstractOuterJoinTaskTest(ExecutionConfig config) {
 		super(config, HASH_MEM, NUM_SORTER, SORT_MEM);
 		bnljn_frac = (double) BNLJN_MEM / this.getMemoryManager().getMemorySize();
-		this.driverStrategy = driverStrategy;
 	}
 	
 	@Test
@@ -162,7 +159,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -191,7 +188,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -220,7 +217,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -249,7 +246,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -266,7 +263,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		
 		this.outList.clear();
 	}
-	
+
 	@Test(expected = ExpectedTestException.class)
 	public void testFailingOuterJoinTask() throws Exception {
 		int keyCnt1 = 20;
@@ -279,7 +276,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -297,7 +294,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -324,7 +321,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		
 		cancel();
 		taskRunner.interrupt();
-		
+
 		taskRunner.join(60000);
 		
 		assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive());
@@ -341,7 +338,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(this.driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(this.bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -385,7 +382,7 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 		addDriverComparator(this.comparator1);
 		addDriverComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
-		getTaskConfig().setDriverStrategy(driverStrategy);
+		getTaskConfig().setDriverStrategy(this.getSortDriverStrategy());
 		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
@@ -426,6 +423,8 @@ public abstract class AbstractOuterJoinTaskTest extends BinaryOperatorTestBase<F
 	protected abstract AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver();
 	
 	protected abstract int calculateExpectedCount(int keyCnt1, int valCnt1, int keyCnt2, int valCnt2);
+
+	protected abstract DriverStrategy getSortDriverStrategy();
 	
 	// =================================================================================================
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskExternalITCase.java
index d52f5fb..15881f1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskExternalITCase.java
@@ -26,7 +26,7 @@ public class FullOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExtern
 	
 	
 	public FullOuterJoinTaskExternalITCase(ExecutionConfig config) {
-		super(config, DriverStrategy.FULL_OUTER_MERGE);
+		super(config);
 	}
 	
 	@Override
@@ -38,4 +38,9 @@ public class FullOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExtern
 	protected AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver() {
 		return new FullOuterJoinDriver<>();
 	}
+
+	@Override
+	protected DriverStrategy getSortStrategy() {
+		return DriverStrategy.FULL_OUTER_MERGE;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskTest.java
index d3296f6..2efedac 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/FullOuterJoinTaskTest.java
@@ -26,7 +26,12 @@ public class FullOuterJoinTaskTest extends AbstractOuterJoinTaskTest {
 	
 	
 	public FullOuterJoinTaskTest(ExecutionConfig config) {
-		super(config, DriverStrategy.FULL_OUTER_MERGE);
+		super(config);
+	}
+
+	@Override
+	protected DriverStrategy getSortDriverStrategy() {
+		return DriverStrategy.FULL_OUTER_MERGE;
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskExternalITCase.java
index 89d68f2..33552bd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskExternalITCase.java
@@ -21,12 +21,18 @@ package org.apache.flink.runtime.operators;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory;
+import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
+import org.junit.Assert;
+import org.junit.Test;
 
 public class LeftOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExternalITCase {
-	
+
+	private final double hash_frac;
 	
 	public LeftOuterJoinTaskExternalITCase(ExecutionConfig config) {
-		super(config, DriverStrategy.LEFT_OUTER_MERGE);
+		super(config);
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Override
@@ -38,4 +44,36 @@ public class LeftOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExtern
 	protected AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver() {
 		return new LeftOuterJoinDriver<>();
 	}
+
+	@Override
+	protected DriverStrategy getSortStrategy() {
+		return DriverStrategy.LEFT_OUTER_MERGE;
+	}
+
+	@Test
+	public void testExternalHashLeftOuterJoinTask() throws Exception {
+
+		final int keyCnt1 = 65536;
+		final int valCnt1 = 8;
+
+		final int keyCnt2 = 32768;
+		final int valCnt2 = 8;
+
+		final int expCnt = calculateExpectedCount(keyCnt1, valCnt1, keyCnt2, valCnt2);
+
+		setOutput(this.output);
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInputSorted(new UniformIntTupleGenerator(keyCnt1, valCnt1, false), serializer, this.comparator1.duplicate());
+		addInputSorted(new UniformIntTupleGenerator(keyCnt2, valCnt2, false), serializer, this.comparator2.duplicate());
+		testDriver(testTask, MockJoinStub.class);
+
+		Assert.assertEquals("Wrong result set size.", expCnt, this.output.getNumberOfRecords());
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java
index 9a1ec8f..ad11768 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java
@@ -19,23 +19,245 @@
 
 package org.apache.flink.runtime.operators;
 
+import com.google.common.base.Throwables;
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.record.RecordPairComparatorFactory;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory;
+import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator;
+import org.apache.flink.runtime.operators.testutils.DelayingIterator;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.ExpectedTestException;
+import org.apache.flink.runtime.operators.testutils.InfiniteIntTupleIterator;
+import org.apache.flink.runtime.operators.testutils.NirvanaOutputList;
+import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
+import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator;
+import org.apache.flink.types.Record;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 public class LeftOuterJoinTaskTest extends AbstractOuterJoinTaskTest {
-	
-	
+
+	private static final long HASH_MEM = 6*1024*1024;
+
+	private final double hash_frac;
+
 	public LeftOuterJoinTaskTest(ExecutionConfig config) {
-		super(config, DriverStrategy.LEFT_OUTER_MERGE);
+		super(config);
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Override
 	protected int calculateExpectedCount(int keyCnt1, int valCnt1, int keyCnt2, int valCnt2) {
 		return valCnt1 * valCnt2 * Math.min(keyCnt1, keyCnt2) + (keyCnt1 > keyCnt2 ? (keyCnt1 - keyCnt2) * valCnt1 : 0);
 	}
-	
+
+	@Override
+	protected DriverStrategy getSortDriverStrategy() {
+		return DriverStrategy.LEFT_OUTER_MERGE;
+	}
+
 	@Override
 	protected AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver() {
 		return new LeftOuterJoinDriver<>();
 	}
+
+	@Test
+	public void testHash1LeftOuterJoinTask() throws Exception {
+		final int keyCnt1 = 20;
+		final int valCnt1 = 1;
+
+		final int keyCnt2 = 10;
+		final int valCnt2 = 2;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash2LeftOuterJoinTask() throws Exception {
+		final int keyCnt1 = 20;
+		final int valCnt1 = 1;
+
+		final int keyCnt2 = 20;
+		final int valCnt2 = 1;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash3LeftOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 1;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash4LeftOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 1;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash5LeftOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash6LeftOuterJoinTask() throws Exception {
+		int keyCnt1 = 10;
+		int valCnt1 = 1;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 2;
+
+		testHashLeftOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	private void testHashLeftOuterJoinTask(int keyCnt1, int valCnt1, int keyCnt2, int valCnt2) throws Exception {
+
+		setOutput(this.outList, this.serializer);
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(keyCnt1, valCnt1, false), this.serializer);
+		addInput(new UniformIntTupleGenerator(keyCnt2, valCnt2, false), this.serializer);
+		testDriver(testTask, MockJoinStub.class);
+
+		final int expCnt = calculateExpectedCount(keyCnt1, valCnt1, keyCnt2, valCnt2);
+
+		Assert.assertTrue("Result set size was " + this.outList.size() + ". Expected was " + expCnt, this.outList.size() == expCnt);
+
+		this.outList.clear();
+	}
+
+	@Test(expected = ExpectedTestException.class)
+	public void testFailingHashLeftOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(keyCnt1, valCnt1, true), this.serializer);
+		addInput(new UniformIntTupleGenerator(keyCnt2, valCnt2, true), this.serializer);
+
+		testDriver(testTask, MockFailingJoinStub.class);
+	}
+
+	@Test
+	public void testCancelLeftOuterJoinTaskWhileBuilding() throws Exception {
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(100, 100, true), this.serializer);
+		addInput(new DelayingIterator<>(new InfiniteIntTupleIterator(), 100), this.serializer);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Thread taskRunner = new Thread("Task runner for testCancelOuterJoinTaskWhileSort1()") {
+			@Override
+			public void run() {
+				try {
+					testDriver(testTask, MockJoinStub.class);
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		taskRunner.start();
+
+		Thread.sleep(1000);
+
+		cancel();
+		taskRunner.join(60000);
+
+		assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive());
+
+		final Throwable taskError = error.get();
+		if (taskError != null) {
+			fail("Error in task while canceling:\n" + Throwables.getStackTraceAsString(taskError));
+		}
+	}
+
+	@Test
+	public void testCancelLeftOuterJoinTaskWhileProbing() throws Exception {
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new DelayingIterator<>(new InfiniteIntTupleIterator(), 100), this.serializer);
+		addInput(new UniformIntTupleGenerator(1, 1, true), this.serializer);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Thread taskRunner = new Thread("Task runner for testCancelOuterJoinTaskWhileSort1()") {
+			@Override
+			public void run() {
+				try {
+					testDriver(testTask, MockJoinStub.class);
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		taskRunner.start();
+
+		Thread.sleep(1000);
+
+		cancel();
+		taskRunner.join(60000);
+
+		assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive());
+
+		final Throwable taskError = error.get();
+		if (taskError != null) {
+			fail("Error in task while canceling:\n" + Throwables.getStackTraceAsString(taskError));
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskExternalITCase.java
index 4e7df4b..ef725ce 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskExternalITCase.java
@@ -21,12 +21,18 @@ package org.apache.flink.runtime.operators;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory;
+import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
+import org.junit.Assert;
+import org.junit.Test;
 
 public class RightOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExternalITCase {
-	
-	
+
+	private final double hash_frac;
+
 	public RightOuterJoinTaskExternalITCase(ExecutionConfig config) {
-		super(config, DriverStrategy.RIGHT_OUTER_MERGE);
+		super(config);
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Override
@@ -38,4 +44,36 @@ public class RightOuterJoinTaskExternalITCase extends AbstractOuterJoinTaskExter
 	protected AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver() {
 		return new RightOuterJoinDriver<>();
 	}
+
+	@Override
+	protected DriverStrategy getSortStrategy() {
+		return DriverStrategy.RIGHT_OUTER_MERGE;
+	}
+
+	@Test
+	public void testExternalHashRightOuterJoinTask() throws Exception {
+
+		final int keyCnt1 = 32768;
+		final int valCnt1 = 8;
+
+		final int keyCnt2 = 65536;
+		final int valCnt2 = 8;
+
+		final int expCnt = calculateExpectedCount(keyCnt1, valCnt1, keyCnt2, valCnt2);
+
+		setOutput(this.output);
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInputSorted(new UniformIntTupleGenerator(keyCnt1, valCnt1, false), serializer, this.comparator1.duplicate());
+		addInputSorted(new UniformIntTupleGenerator(keyCnt2, valCnt2, false), serializer, this.comparator2.duplicate());
+		testDriver(testTask, MockJoinStub.class);
+
+		Assert.assertEquals("Wrong result set size.", expCnt, this.output.getNumberOfRecords());
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java
index 506e95b..4d41031 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java
@@ -19,14 +19,37 @@
 
 package org.apache.flink.runtime.operators;
 
+import com.google.common.base.Throwables;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory;
+import org.apache.flink.runtime.operators.testutils.DelayingIterator;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.ExpectedTestException;
+import org.apache.flink.runtime.operators.testutils.InfiniteIntTupleIterator;
+import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 public class RightOuterJoinTaskTest extends AbstractOuterJoinTaskTest {
-	
+
+	private static final long HASH_MEM = 6*1024*1024;
+
+	private final double hash_frac;
 	
 	public RightOuterJoinTaskTest(ExecutionConfig config) {
-		super(config, DriverStrategy.RIGHT_OUTER_MERGE);
+		super(config);
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
+	}
+
+	@Override
+	protected DriverStrategy getSortDriverStrategy() {
+		return DriverStrategy.RIGHT_OUTER_MERGE;
 	}
 	
 	@Override
@@ -38,4 +61,200 @@ public class RightOuterJoinTaskTest extends AbstractOuterJoinTaskTest {
 	protected AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> getOuterJoinDriver() {
 		return new RightOuterJoinDriver<>();
 	}
+
+	@Test
+	public void testHash1RightOuterJoinTask() throws Exception {
+		final int keyCnt1 = 20;
+		final int valCnt1 = 1;
+
+		final int keyCnt2 = 10;
+		final int valCnt2 = 2;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash2RightOuterJoinTask() throws Exception {
+		final int keyCnt1 = 20;
+		final int valCnt1 = 1;
+
+		final int keyCnt2 = 20;
+		final int valCnt2 = 1;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash3RightOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 1;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash4RightOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 1;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash5RightOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	@Test
+	public void testHash6RightOuterJoinTask() throws Exception {
+		int keyCnt1 = 10;
+		int valCnt1 = 1;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 2;
+
+		testHashRightOuterJoinTask(keyCnt1, valCnt1, keyCnt2, valCnt2);
+	}
+
+	private void testHashRightOuterJoinTask(int keyCnt1, int valCnt1, int keyCnt2, int valCnt2) throws Exception {
+
+		setOutput(this.outList, this.serializer);
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
+
+		setNumFileHandlesForSort(4);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(keyCnt1, valCnt1, false), this.serializer);
+		addInput(new UniformIntTupleGenerator(keyCnt2, valCnt2, false), this.serializer);
+		testDriver(testTask, MockJoinStub.class);
+
+		final int expCnt = calculateExpectedCount(keyCnt1, valCnt1, keyCnt2, valCnt2);
+
+		Assert.assertTrue("Result set size was " + this.outList.size() + ". Expected was " + expCnt, this.outList.size() == expCnt);
+
+		this.outList.clear();
+	}
+
+	@Test(expected = ExpectedTestException.class)
+	public void testFailingHashRightOuterJoinTask() throws Exception {
+		int keyCnt1 = 20;
+		int valCnt1 = 20;
+
+		int keyCnt2 = 20;
+		int valCnt2 = 20;
+
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+		setNumFileHandlesForSort(4);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(keyCnt1, valCnt1, true), this.serializer);
+		addInput(new UniformIntTupleGenerator(keyCnt2, valCnt2, true), this.serializer);
+
+		testDriver(testTask, MockFailingJoinStub.class);
+	}
+
+	@Test
+	public void testCancelRightOuterJoinTaskWhileBuilding() throws Exception {
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new DelayingIterator<>(new InfiniteIntTupleIterator(), 100), this.serializer);
+		addInput(new UniformIntTupleGenerator(100, 100, true), this.serializer);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Thread taskRunner = new Thread("Task runner for testCancelOuterJoinTaskWhileSort1()") {
+			@Override
+			public void run() {
+				try {
+					testDriver(testTask, MockJoinStub.class);
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		taskRunner.start();
+
+		Thread.sleep(1000);
+
+		cancel();
+		taskRunner.join(60000);
+
+		assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive());
+
+		final Throwable taskError = error.get();
+		if (taskError != null) {
+			fail("Error in task while canceling:\n" + Throwables.getStackTraceAsString(taskError));
+		}
+	}
+
+	@Test
+	public void testCancelRightOuterJoinTaskWhileProbing() throws Exception {
+		setOutput(new DiscardingOutputCollector<Tuple2<Integer, Integer>>());
+		addDriverComparator(this.comparator1);
+		addDriverComparator(this.comparator2);
+		getTaskConfig().setDriverPairComparator(new RuntimePairComparatorFactory());
+		getTaskConfig().setDriverStrategy(DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST);
+		getTaskConfig().setRelativeMemoryDriver(this.hash_frac);
+
+		final AbstractOuterJoinDriver<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testTask = getOuterJoinDriver();
+
+		addInput(new UniformIntTupleGenerator(1, 1, true), this.serializer);
+		addInput(new DelayingIterator<>(new InfiniteIntTupleIterator(), 100), this.serializer);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Thread taskRunner = new Thread("Task runner for testCancelOuterJoinTaskWhileSort1()") {
+			@Override
+			public void run() {
+				try {
+					testDriver(testTask, MockJoinStub.class);
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		taskRunner.start();
+
+		Thread.sleep(1000);
+
+		cancel();
+		taskRunner.join(60000);
+
+		assertFalse("Task thread did not finish within 60 seconds", taskRunner.isAlive());
+
+		final Throwable taskError = error.get();
+		if (taskError != null) {
+			fail("Error in task while canceling:\n" + Throwables.getStackTraceAsString(taskError));
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
new file mode 100644
index 0000000..a766214
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
@@ -0,0 +1,947 @@
+/*
+ * 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.runtime.operators.hash;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+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.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.TestData;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
+import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
+import org.apache.flink.runtime.operators.testutils.UnionIterator;
+import org.apache.flink.runtime.operators.testutils.types.IntPair;
+import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer;
+import org.apache.flink.types.NullKeyFieldException;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.flink.api.common.typeutils.GenericPairComparator;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+@SuppressWarnings({"serial", "deprecation"})
+public class NonReusingHashJoinIteratorITCase {
+	
+	private static final int MEMORY_SIZE = 16000000;		// total memory
+
+	private static final int INPUT_1_SIZE = 20000;
+	private static final int INPUT_2_SIZE = 1000;
+
+	private static final long SEED1 = 561349061987311L;
+	private static final long SEED2 = 231434613412342L;
+	
+	private final AbstractInvokable parentTask = new DummyInvokable();
+
+	private IOManager ioManager;
+	private MemoryManager memoryManager;
+	
+	private TypeSerializer<Tuple2<Integer, String>> recordSerializer;
+	private TypeComparator<Tuple2<Integer, String>> record1Comparator;
+	private TypeComparator<Tuple2<Integer, String>> record2Comparator;
+	private TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> recordPairComparator;
+	
+	private TypeSerializer<IntPair> pairSerializer;
+	private TypeComparator<IntPair> pairComparator;
+	private TypePairComparator<IntPair, Tuple2<Integer, String>> pairRecordPairComparator;
+	private TypePairComparator<Tuple2<Integer, String>, IntPair> recordPairPairComparator;
+
+
+	@SuppressWarnings("unchecked")
+	@Before
+	public void beforeTest() {
+		this.recordSerializer = TestData.getIntStringTupleSerializer();
+		
+		this.record1Comparator = TestData.getIntStringTupleComparator();
+		this.record2Comparator = TestData.getIntStringTupleComparator();
+		
+		this.recordPairComparator = new GenericPairComparator(record1Comparator, record2Comparator);
+		
+		this.pairSerializer = new IntPairSerializer();
+		this.pairComparator = new TestData.IntPairComparator();
+		this.pairRecordPairComparator = new IntPairTuplePairComparator();
+		this.recordPairPairComparator = new TupleIntPairPairComparator();
+		
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
+		this.ioManager = new IOManagerAsync();
+	}
+
+	@After
+	public void afterTest() {
+		if (this.ioManager != null) {
+			this.ioManager.shutdown();
+			if (!this.ioManager.isProperlyShutDown()) {
+				Assert.fail("I/O manager failed to properly shut down.");
+			}
+			this.ioManager = null;
+		}
+		
+		if (this.memoryManager != null) {
+			Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.",
+				this.memoryManager.verifyEmpty());
+			this.memoryManager.shutdown();
+			this.memoryManager = null;
+		}
+	}
+
+
+	@Test
+	public void testBuildFirst() {
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<Tuple2<Integer, String>>();
+	
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			NonReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.recordSerializer, this.record1Comparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+						this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildFirstWithHighNumberOfCommonKeys()
+	{
+		// the size of the left and right inputs
+		final int INPUT_1_SIZE = 200;
+		final int INPUT_2_SIZE = 100;
+		
+		final int INPUT_1_DUPLICATES = 10;
+		final int INPUT_2_DUPLICATES = 2000;
+		final int DUPLICATE_KEY = 13;
+		
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
+			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+			
+			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
+			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
+			
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			// re-create the whole thing for actual processing
+			
+			// reset the generators and iterators
+			generator1.reset();
+			generator2.reset();
+			const1Iter.reset();
+			const2Iter.reset();
+			gen1Iter.reset();
+			gen2Iter.reset();
+			
+			inList1.clear();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			inList2.clear();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+	
+			input1 = new UnionIterator<>(inList1);
+			input2 = new UnionIterator<>(inList2);
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			NonReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.recordSerializer, this.record1Comparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+						this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecond() {
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+	
+			// compare with iterator values			
+			NonReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+				new NonReusingBuildSecondHashJoinIterator<>(
+					input1, input2, this.recordSerializer, this.record1Comparator, 
+					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecondWithHighNumberOfCommonKeys()
+	{
+		// the size of the left and right inputs
+		final int INPUT_1_SIZE = 200;
+		final int INPUT_2_SIZE = 100;
+		
+		final int INPUT_1_DUPLICATES = 10;
+		final int INPUT_2_DUPLICATES = 2000;
+		final int DUPLICATE_KEY = 13;
+		
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
+			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+			
+			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
+			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
+			
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			// re-create the whole thing for actual processing
+			
+			// reset the generators and iterators
+			generator1.reset();
+			generator2.reset();
+			const1Iter.reset();
+			const2Iter.reset();
+			gen1Iter.reset();
+			gen2Iter.reset();
+			
+			inList1.clear();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			inList2.clear();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+	
+			input1 = new UnionIterator<>(inList1);
+			input2 = new UnionIterator<>(inList2);
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+
+			NonReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+				new NonReusingBuildSecondHashJoinIterator<>(
+					input1, input2, this.recordSerializer, this.record1Comparator, 
+					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildFirstWithMixedDataTypes() {
+		try {
+			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
+			
+			final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = joinIntPairs(
+					collectIntPairData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			input1 = new UniformIntPairGenerator(500, 40, false);
+			generator2.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			NonReusingBuildSecondHashJoinIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildSecondHashJoinIterator<>(
+						input1, input2, this.pairSerializer, this.pairComparator,
+						this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator,
+						this.memoryManager, this.ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecondWithMixedDataTypes() {
+		try {
+			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
+			
+			final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = joinIntPairs(
+					collectIntPairData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			input1 = new UniformIntPairGenerator(500, 40, false);
+			generator2.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			NonReusingBuildFirstHashJoinIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.pairSerializer, this.pairComparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairPairComparator,
+						this.memoryManager, this.ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBuildFirstJoinWithEmptyBuild() {
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 1000, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = rightOuterJoinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+
+			// compare with iterator values
+			NonReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildFirstHashJoinIterator<>(
+							input1, input2, this.recordSerializer, this.record1Comparator,
+							this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+							this.memoryManager, ioManager, this.parentTask, 1.0, true, false);
+
+			iterator.open();
+
+			while (iterator.callWithNextKey(matcher, collector));
+
+			iterator.close();
+
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBuildSecondJoinWithEmptyBuild() {
+		try {
+			TupleGenerator generator1 = new TupleGenerator(SEED1, 1000, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = leftOuterJoinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+
+			// compare with iterator values
+			NonReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new NonReusingBuildSecondHashJoinIterator<>(
+							input1, input2, this.recordSerializer, this.record1Comparator,
+							this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+							this.memoryManager, ioManager, this.parentTask, 1.0, true, false);
+
+			iterator.open();
+
+			while (iterator.callWithNextKey(matcher, collector));
+
+			iterator.close();
+
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                    Utilities
+	// --------------------------------------------------------------------------------------------
+
+	
+	
+	public static Map<Integer, Collection<TupleMatch>> joinTuples(
+			Map<Integer, Collection<String>> leftMap,
+			Map<Integer, Collection<String>> rightMap)
+	{
+		Map<Integer, Collection<TupleMatch>> map = new HashMap<>();
+
+		for (Integer key : leftMap.keySet()) {
+			Collection<String> leftValues = leftMap.get(key);
+			Collection<String> rightValues = rightMap.get(key);
+
+			if (rightValues == null) {
+				continue;
+			}
+
+			if (!map.containsKey(key)) {
+				map.put(key, new ArrayList<TupleMatch>());
+			}
+
+			Collection<TupleMatch> matchedValues = map.get(key);
+
+			for (String leftValue : leftValues) {
+				for (String rightValue : rightValues) {
+					matchedValues.add(new TupleMatch(leftValue, rightValue));
+				}
+			}
+		}
+
+		return map;
+	}
+
+	public static Map<Integer, Collection<TupleMatch>> leftOuterJoinTuples(
+			Map<Integer, Collection<String>> leftMap,
+			Map<Integer, Collection<String>> rightMap)
+	{
+		Map<Integer, Collection<TupleMatch>> map = new HashMap<>();
+
+		for (Integer key : leftMap.keySet()) {
+			Collection<String> leftValues = leftMap.get(key);
+			Collection<String> rightValues = rightMap.get(key);
+
+			if (!map.containsKey(key)) {
+				map.put(key, new ArrayList<TupleMatch>());
+			}
+
+			Collection<TupleMatch> matchedValues = map.get(key);
+
+			for (String leftValue : leftValues) {
+				if(rightValues != null) {
+					for (String rightValue : rightValues) {
+						matchedValues.add(new TupleMatch(leftValue, rightValue));
+					}
+				}
+				else {
+					matchedValues.add(new TupleMatch(leftValue, null));
+				}
+			}
+		}
+
+		return map;
+	}
+
+	public static Map<Integer, Collection<TupleMatch>> rightOuterJoinTuples(
+			Map<Integer, Collection<String>> leftMap,
+			Map<Integer, Collection<String>> rightMap)
+	{
+		Map<Integer, Collection<TupleMatch>> map = new HashMap<>();
+
+		for (Integer key : rightMap.keySet()) {
+			Collection<String> leftValues = leftMap.get(key);
+			Collection<String> rightValues = rightMap.get(key);
+
+			if (!map.containsKey(key)) {
+				map.put(key, new ArrayList<TupleMatch>());
+			}
+
+			Collection<TupleMatch> matchedValues = map.get(key);
+
+			for (String rightValue : rightValues) {
+				if(leftValues != null) {
+					for (String leftValue : leftValues) {
+						matchedValues.add(new TupleMatch(leftValue, rightValue));
+					}
+				}
+				else {
+					matchedValues.add(new TupleMatch(null, rightValue));
+				}
+			}
+		}
+
+		return map;
+	}
+	
+	public static Map<Integer, Collection<TupleIntPairMatch>> joinIntPairs(
+			Map<Integer, Collection<Integer>> leftMap,
+			Map<Integer, Collection<String>> rightMap)
+	{
+		final Map<Integer, Collection<TupleIntPairMatch>> map = new HashMap<>();
+	
+		for (Integer i : leftMap.keySet()) {
+			
+			final Collection<Integer> leftValues = leftMap.get(i);
+			final Collection<String> rightValues = rightMap.get(i);
+	
+			if (rightValues == null) {
+				continue;
+			}
+	
+			if (!map.containsKey(i)) {
+				map.put(i, new ArrayList<TupleIntPairMatch>());
+			}
+	
+			final Collection<TupleIntPairMatch> matchedValues = map.get(i);
+	
+			for (Integer v : leftValues) {
+				for (String val : rightValues) {
+					matchedValues.add(new TupleIntPairMatch(v, val));
+				}
+			}
+		}
+	
+		return map;
+	}
+
+	
+	public static Map<Integer, Collection<String>> collectTupleData(MutableObjectIterator<Tuple2<Integer, String>> iter)
+	throws Exception
+	{
+		Map<Integer, Collection<String>> map = new HashMap<>();
+		Tuple2<Integer, String> pair = new Tuple2<>();
+		
+		while ((pair = iter.next(pair)) != null) {
+
+			Integer key = pair.f0;
+			if (!map.containsKey(key)) {
+				map.put(key, new ArrayList<String>());
+			}
+
+			Collection<String> values = map.get(key);
+			values.add(pair.f1);
+		}
+
+		return map;
+	}
+	
+	public static Map<Integer, Collection<Integer>> collectIntPairData(MutableObjectIterator<IntPair> iter)
+	throws Exception
+	{
+		Map<Integer, Collection<Integer>> map = new HashMap<>();
+		IntPair pair = new IntPair();
+		
+		while ((pair = iter.next(pair)) != null) {
+
+			final int key = pair.getKey();
+			final int value = pair.getValue();
+			if (!map.containsKey(key)) {
+				map.put(key, new ArrayList<Integer>());
+			}
+
+			Collection<Integer> values = map.get(key);
+			values.add(value);
+		}
+
+		return map;
+	}
+
+	/**
+	 * Class used for storage of the expected matches in a hash-map.
+	 */
+	public static class TupleMatch {
+		
+		private final String left;
+		private final String right;
+
+		public TupleMatch(String left, String right) {
+			this.left = left;
+			this.right = right;
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			TupleMatch o = (TupleMatch) obj;
+
+			if(left != null && o.left != null && right != null && o.right != null) {
+				return this.left.equals(o.left) && this.right.equals(o.right);
+			}
+			else if(left == null && o.left == null) {
+				return this.right.equals(o.right);
+			}
+			else if(right == null && o.right == null) {
+				return this.left.equals(o.left);
+			}
+			else if(left == null && o.left == null && right == null && o.right == null) {
+				return true;
+			}
+			else {
+				return false;
+			}
+		}
+		
+		@Override
+		public int hashCode() {
+			int hc = this.left != null ? this.left.hashCode() : 23;
+			hc = hc ^ (this.right != null ? this.right.hashCode() : 41);
+			return hc;
+		}
+
+		@Override
+		public String toString() {
+			String s = left == null ? "<null>" : left;
+			s += ", " + right == null ? "<null>" : right;
+			return s;
+		}
+	}
+	
+	/**
+	 * Private class used for storage of the expected matches in a hash-map.
+	 */
+	public static class TupleIntPairMatch
+	{
+		private final int left;
+		private final String right;
+
+		public TupleIntPairMatch(int left, String right) {
+			this.left = left;
+			this.right = new String(right);
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			TupleIntPairMatch o = (TupleIntPairMatch) obj;
+			return this.left == o.left && this.right.equals(o.right);
+		}
+		
+		@Override
+		public int hashCode() {
+			return this.left ^ this.right.hashCode();
+		}
+
+		@Override
+		public String toString() {
+			return left + ", " + right;
+		}
+	}
+	
+	static final class TupleMatchRemovingJoin implements FlatJoinFunction<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
+	{
+		private final Map<Integer, Collection<TupleMatch>> toRemoveFrom;
+		
+		protected TupleMatchRemovingJoin(Map<Integer, Collection<TupleMatch>> map) {
+			this.toRemoveFrom = map;
+		}
+		
+		@Override
+		public void join(Tuple2<Integer, String> rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
+		{
+
+			int key = rec1 != null ? rec1.f0 : rec2.f0;
+			String value1 = rec1 != null ? rec1.f1 : null;
+			String value2 = rec2 != null ? rec2.f1 : null;
+
+			//System.err.println("rec1 key = "+key+"  rec2 key= "+rec2.f0);
+			Collection<TupleMatch> matches = this.toRemoveFrom.get(key);
+			if (matches == null) {
+				Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected.");
+			}
+			
+			Assert.assertTrue("Produced match was not contained: " + key + " - " + value1 + ":" + value2,
+				matches.remove(new TupleMatch(value1, value2)));
+			
+			if (matches.isEmpty()) {
+				this.toRemoveFrom.remove(key);
+			}
+		}
+	}
+	
+	static final class TupleIntPairMatchRemovingMatcher implements FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>>
+	{
+		private final Map<Integer, Collection<TupleIntPairMatch>> toRemoveFrom;
+		
+		protected TupleIntPairMatchRemovingMatcher(Map<Integer, Collection<TupleIntPairMatch>> map) {
+			this.toRemoveFrom = map;
+		}
+		
+		@Override
+		public void join(IntPair rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
+		{
+			final int k = rec1.getKey();
+			final int v = rec1.getValue(); 
+			
+			final Integer key = rec2.f0;
+			final String value = rec2.f1;
+
+			Assert.assertTrue("Key does not match for matching IntPair Tuple combination.", k == key);
+			
+			Collection<TupleIntPairMatch> matches = this.toRemoveFrom.get(key);
+			if (matches == null) {
+				Assert.fail("Match " + key + " - " + v + ":" + value + " is unexpected.");
+			}
+			
+			Assert.assertTrue("Produced match was not contained: " + key + " - " + v + ":" + value,
+				matches.remove(new TupleIntPairMatch(v, value)));
+			
+			if (matches.isEmpty()) {
+				this.toRemoveFrom.remove(key);
+			}
+		}
+	}
+	
+	static final class IntPairTuplePairComparator extends TypePairComparator<IntPair, Tuple2<Integer, String>>
+	{
+		private int reference;
+		
+		@Override
+		public void setReference(IntPair reference) {
+			this.reference = reference.getKey();	
+		}
+
+		@Override
+		public boolean equalToReference(Tuple2<Integer, String> candidate) {
+			try {
+				return candidate.f0 == this.reference;
+			} catch (NullPointerException npex) {
+				throw new NullKeyFieldException();
+			}
+		}
+
+		@Override
+		public int compareToReference(Tuple2<Integer, String> candidate) {
+			try {
+				return candidate.f0 - this.reference;
+			} catch (NullPointerException npex) {
+				throw new NullKeyFieldException();
+			}
+		}
+	}
+	
+	static final class TupleIntPairPairComparator extends TypePairComparator<Tuple2<Integer, String>, IntPair>
+	{
+		private int reference;
+		
+		@Override
+		public void setReference(Tuple2<Integer, String> reference) {
+			this.reference = reference.f0;
+		}
+
+		@Override
+		public boolean equalToReference(IntPair candidate) {
+			return this.reference == candidate.getKey();
+		}
+
+		@Override
+		public int compareToReference(IntPair candidate) {
+			return candidate.getKey() - this.reference;
+		}
+	}
+}


[05/10] flink git commit: [FLINK-2842] [documentation] Remove Flink S3FileSystem, extend documentation to use Hadoop S3FileSystem.

Posted by fh...@apache.org.
[FLINK-2842] [documentation] Remove Flink S3FileSystem, extend documentation to use Hadoop S3FileSystem.

This closes #1245


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

Branch: refs/heads/master
Commit: c90057792d887d3ab7ff23ff53149f1887c9ed62
Parents: 640e63b
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Oct 9 17:35:00 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 19 16:17:37 2015 +0200

----------------------------------------------------------------------
 docs/apis/example_connectors.md                 |  68 +-
 .../org/apache/flink/core/fs/FileSystem.java    |   3 -
 flink-runtime/pom.xml                           |  12 +-
 .../flink/runtime/fs/s3/S3BlockLocation.java    |  63 --
 .../flink/runtime/fs/s3/S3BucketObjectPair.java |  93 ---
 .../flink/runtime/fs/s3/S3DataInputStream.java  | 174 ----
 .../flink/runtime/fs/s3/S3DataOutputStream.java | 325 --------
 .../runtime/fs/s3/S3DirectoryStructure.java     |  87 --
 .../flink/runtime/fs/s3/S3FileStatus.java       |  95 ---
 .../flink/runtime/fs/s3/S3FileSystem.java       | 786 -------------------
 .../flink/runtime/fs/s3/S3FileSystemTest.java   | 465 -----------
 11 files changed, 52 insertions(+), 2119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/docs/apis/example_connectors.md
----------------------------------------------------------------------
diff --git a/docs/apis/example_connectors.md b/docs/apis/example_connectors.md
index 5b3f7c6..ef5e994 100644
--- a/docs/apis/example_connectors.md
+++ b/docs/apis/example_connectors.md
@@ -20,32 +20,64 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-## Reading from filesystems.
+## Reading from file systems.
 
 Flink has build-in support for the following file systems:
 
-| Filesystem        | Since           | Scheme  | Notes |
-| ------------- |-------------| -----| ------ |
-| Hadoop Distributed File System (HDFS)  | 0.2 | `hdfs://`| All HDFS versions are supported |
-| Amazon S3    |  0.2 | `s3://` |   |
-| MapR file system      | 0.7-incubating      |  `maprfs://` | The user has to manually place the required jar files in the `lib/` dir |
-| Tachyon   |  0.9 | `tachyon://` | Support through Hadoop file system implementation (see below) |
+| Filesystem                            | Scheme       | Notes  |
+| ------------------------------------- |--------------| ------ |
+| Hadoop Distributed File System (HDFS) &nbsp; | `hdfs://`    | All HDFS versions are supported |
+| Amazon S3                             | `s3://`      | Support through Hadoop file system implementation (see below) | 
+| MapR file system                      | `maprfs://`  | The user has to manually place the required jar files in the `lib/` dir |
+| Tachyon                               | `tachyon://` &nbsp; | Support through Hadoop file system implementation (see below) |
 
 
 
-### Using Hadoop file systems with Apache Flink
+### Using Hadoop file system implementations
 
 Apache Flink allows users to use any file system implementing the `org.apache.hadoop.fs.FileSystem`
-interface. Hadoop ships adapters for FTP, [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html), and others.
+interface. There are Hadoop `FileSystem` implementations for
 
-Flink has integrated testcases to validate the integration with [Tachyon](http://tachyon-project.org/).
-Other file systems we tested the integration is the
-[Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector) and [XtreemFS](http://www.xtreemfs.org/).
+- [S3](https://aws.amazon.com/s3/) (tested)
+- [Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector) (tested)
+- [Tachyon](http://tachyon-project.org/) (tested)
+- [XtreemFS](http://www.xtreemfs.org/) (tested)
+- FTP via [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html) (not tested)
+- and many more.
 
-In order to use a Hadoop file system with Flink, make sure that the `flink-conf.yaml` has set the
-`fs.hdfs.hadoopconf` property set to the Hadoop configuration directory.
-In addition to that, the Hadoop configuration (in that directory) needs to have an entry for each supported file system.
-For example for tachyon support, there must be the following entry in the `core-site.xml` file:
+In order to use a Hadoop file system with Flink, make sure that
+
+- the `flink-conf.yaml` has set the `fs.hdfs.hadoopconf` property set to the Hadoop configuration directory.
+- the Hadoop configuration (in that directory) has an entry for the required file system. Examples for S3 and Tachyon are shown below.
+- the required classes for using the file system are available in the `lib/` folder of the Flink installation (on all machines running Flink). If putting the files into the directory is not possible, Flink is also respecting the `HADOOP_CLASSPATH` environment variable to add Hadoop jar files to the classpath.
+
+#### Amazon S3
+
+For Amazon S3 support add the following entries into the `core-site.xml` file:
+
+~~~xml
+<!-- configure the file system implementation -->
+<property>
+  <name>fs.s3.impl</name>
+  <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
+</property>
+
+<!-- set your AWS ID -->
+<property>
+  <name>fs.s3.awsAccessKeyId</name>
+  <value>putKeyHere</value>
+</property>
+
+<!-- set your AWS access key -->
+<property>
+  <name>fs.s3.awsSecretAccessKey</name>
+  <value>putSecretHere</value>
+</property>
+~~~
+
+#### Tachyon
+
+For Tachyon support add the following entry into the `core-site.xml` file:
 
 ~~~xml
 <property>
@@ -54,10 +86,8 @@ For example for tachyon support, there must be the following entry in the `core-
 </property>
 ~~~
 
-Also, the required classes for using the file system need to be placed in the `lib/` folder of the Flink installation (on all machines running Flink). If putting the files into the directory is not possible, Flink is also respecting the `HADOOP_CLASSPATH` environment variable to add Hadoop jar files to the classpath.
-
 
-## Connecting to other systems using Input / Output Format wrappers for Hadoop
+## Connecting to other systems using Input/OutputFormat wrappers for Hadoop
 
 Apache Flink allows users to access many different systems as data sources or sinks.
 The system is designed for very easy extensibility. Similar to Apache Hadoop, Flink has the concept

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index 8b4cdba..185b5f2 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -51,8 +51,6 @@ public abstract class FileSystem {
 
 	private static final String MAPR_FILESYSTEM_CLASS = "org.apache.flink.runtime.fs.maprfs.MapRFileSystem";
 	
-	private static final String S3_FILESYSTEM_CLASS = "org.apache.flink.runtime.fs.s3.S3FileSystem";
-
 	private static final String HADOOP_WRAPPER_SCHEME = "hdwrapper";
 
 	
@@ -156,7 +154,6 @@ public abstract class FileSystem {
 		FSDIRECTORY.put("hdfs", HADOOP_WRAPPER_FILESYSTEM_CLASS);
 		FSDIRECTORY.put("maprfs", MAPR_FILESYSTEM_CLASS);
 		FSDIRECTORY.put("file", LOCAL_FILESYSTEM_CLASS);
-		FSDIRECTORY.put("s3", S3_FILESYSTEM_CLASS);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index a831eba..f79c5ed 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -76,15 +76,9 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>com.amazonaws</groupId>
-			<artifactId>aws-java-sdk</artifactId>
-			<version>1.8.1</version>
-			<exclusions>
-				<exclusion>
-					<artifactId>commons-codec</artifactId>
-					<groupId>commons-codec</groupId>
-				</exclusion>
-			</exclusions>
+			<groupId>org.apache.httpcomponents</groupId>
+			<artifactId>httpclient</artifactId>
+			<version>4.2</version>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BlockLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BlockLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BlockLocation.java
deleted file mode 100644
index 05b716d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BlockLocation.java
+++ /dev/null
@@ -1,63 +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.runtime.fs.s3;
-
-import java.io.IOException;
-
-import org.apache.flink.core.fs.BlockLocation;
-
-public final class S3BlockLocation implements BlockLocation {
-
-	private final String[] hosts;
-
-	private final long length;
-
-	S3BlockLocation(final String host, final long length) {
-
-		this.hosts = new String[1];
-		this.hosts[0] = host;
-		this.length = length;
-	}
-
-	@Override
-	public int compareTo(final BlockLocation arg0) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public String[] getHosts() throws IOException {
-
-		return this.hosts;
-	}
-
-	@Override
-	public long getOffset() {
-
-		return 0;
-	}
-
-	@Override
-	public long getLength() {
-
-		return this.length;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BucketObjectPair.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BucketObjectPair.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BucketObjectPair.java
deleted file mode 100644
index 1f9e0c4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3BucketObjectPair.java
+++ /dev/null
@@ -1,93 +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.runtime.fs.s3;
-
-/**
- * An S3 bucket-object pair identifies either a bucket in S3 or an object. If the object property is <code>null</code>,
- * this object identifies an S3 bucket. If both the bucket and the object property is <code>null</code>, the object
- * refers to the S3 base directory.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class S3BucketObjectPair {
-
-	/**
-	 * The name of the S3 bucket this object refers to.
-	 */
-	private final String bucket;
-
-	/**
-	 * The name of the S3 object this object refers to.
-	 */
-	private final String object;
-
-	/**
-	 * Creates a new S3 bucket-object pair.
-	 * 
-	 * @param bucket
-	 *        the name of the S3 bucket this object refers to
-	 * @param object
-	 *        the name of the S3 object this object refers to
-	 */
-	S3BucketObjectPair(final String bucket, final String object) {
-		this.bucket = bucket;
-		this.object = object;
-	}
-
-	/**
-	 * Returns the name of the S3 bucket this object refers to.
-	 * 
-	 * @return the name of the S3 bucket this object refers to or <code>null</code> if this object refers to the S3 base
-	 *         directory
-	 */
-	public String getBucket() {
-		return this.bucket;
-	}
-
-	/**
-	 * Returns the name of the S3 object this object refers to.
-	 * 
-	 * @return the name of the S3 object this object refers to or <code>null</code> if this object refers to an S3
-	 *         bucket
-	 */
-	public String getObject() {
-		return this.object;
-	}
-
-	/**
-	 * Checks whether this object refers to an S3 bucket.
-	 * 
-	 * @return <code>true</code> if this object refers to an S3 bucket, <code>false</code> otherwise
-	 *         directory
-	 */
-	public boolean hasBucket() {
-		return (this.bucket != null);
-	}
-
-	/**
-	 * Checks whether this object refers to an S3 object.
-	 * 
-	 * @return <code>true</code> if this object refers to an S3 object, <code>false</code> otherwise
-	 */
-	public boolean hasObject() {
-		return (this.object != null);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataInputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataInputStream.java
deleted file mode 100644
index c839640..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataInputStream.java
+++ /dev/null
@@ -1,174 +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.runtime.fs.s3;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.util.StringUtils;
-
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.model.S3Object;
-
-/**
- * This class implements an {@link FSDataInputStream} that downloads its data from Amazon S3 in the background.
- * Essentially, this class is just a wrapper to the Amazon AWS SDK.
- */
-public class S3DataInputStream extends FSDataInputStream {
-
-	/**
-	 * The input stream which reads the actual S3 object content.
-	 */
-	private final InputStream inputStream;
-
-	/**
-	 * The current position of input stream.
-	 */
-	private long position;
-
-	/**
-	 * The marked position.
-	 */
-	private long marked;
-
-
-	/**
-	 * Constructs a new input stream which reads its data from the specified S3 object.
-	 *
-	 * @param s3Client
-	 *        the S3 client to connect to Amazon S3.
-	 * @param bucket
-	 *        the name of the S3 bucket the object is stored in
-	 * @param object
-	 *        the name of the S3 object whose content shall be read
-	 * @throws IOException
-	 *         thrown if an error occurs while accessing the specified S3 object
-	 */
-	S3DataInputStream(final AmazonS3Client s3Client, final String bucket, final String object) throws IOException {
-
-		S3Object s3o = null;
-		try {
-			s3o = s3Client.getObject(bucket, object);
-		} catch (AmazonServiceException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-
-		this.inputStream = s3o.getObjectContent();
-		this.position = 0;
-		this.marked = 0;
-	}
-
-
-	@Override
-	public int available() throws IOException {
-
-		return this.inputStream.available();
-	}
-
-
-	@Override
-	public void close() throws IOException {
-
-		this.inputStream.close();
-	}
-
-
-	@Override
-	public void mark(final int readlimit) {
-
-		this.inputStream.mark(readlimit);
-		marked = readlimit;
-	}
-
-
-	@Override
-	public boolean markSupported() {
-
-		return this.inputStream.markSupported();
-	}
-
-
-	@Override
-	public int read() throws IOException {
-
-		int read = this.inputStream.read();
-		if (read != -1) {
-			++position;
-		}
-
-		return read;
-	}
-
-
-	@Override
-	public int read(final byte[] b) throws IOException {
-
-		int read = this.inputStream.read(b);
-		if (read > 0) {
-			position += read;
-		}
-
-		return read;
-	}
-
-
-	@Override
-	public int read(final byte[] b, final int off, final int len) throws IOException {
-
-		int read = this.inputStream.read(b, off, len);
-		if (read > 0) {
-			position += read;
-		}
-
-		return read;
-	}
-
-
-	@Override
-	public void reset() throws IOException {
-
-		this.inputStream.reset();
-		position = marked;
-	}
-
-
-	@Override
-	public void seek(final long desired) throws IOException {
-
-		skip(desired);
-	}
-
-	@Override
-	public long skip(long n) throws IOException {
-		long skipped = this.inputStream.skip(n);
-		if (skipped > 0) {
-			position += skipped;
-		}
-
-		return skipped;
-	}
-
-	@Override
-	public long getPos() throws IOException {
-		return position;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
deleted file mode 100644
index f830613..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
+++ /dev/null
@@ -1,325 +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.runtime.fs.s3;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.util.StringUtils;
-
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
-import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.PartETag;
-import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.StorageClass;
-import com.amazonaws.services.s3.model.UploadPartRequest;
-import com.amazonaws.services.s3.model.UploadPartResult;
-
-public final class S3DataOutputStream extends FSDataOutputStream {
-
-	private static final int MAX_PART_NUMBER = 10000;
-
-	public static final int MINIMUM_MULTIPART_SIZE = 5 * 1024 * 1024;
-
-	private final AmazonS3Client s3Client;
-
-	private final boolean useRRS;
-
-	private final byte[] buf;
-
-	private final String bucket;
-
-	private final String object;
-
-	private final List<PartETag> partETags = new ArrayList<PartETag>();
-
-	/**
-	 * The ID of a multipart upload in case multipart upload is used, otherwise <code>null</code>.
-	 */
-	private String uploadId = null;
-
-	/**
-	 * The next part number to be used during a multipart upload.
-	 */
-	private int partNumber = 1; // First valid upload part number is 1.
-
-	private int bytesWritten = 0;
-	
-
-	private final class InternalUploadInputStream extends InputStream {
-
-		private final byte[] srcBuf;
-
-		private final int length;
-
-		private int bytesRead = 0;
-
-		private InternalUploadInputStream(final byte[] srcBuf, final int length) {
-			this.srcBuf = buf;
-			this.length = length;
-		}
-
-
-		@Override
-		public int read() throws IOException {
-			if (this.length - this.bytesRead == 0) {
-				return -1;
-			}
-
-			return (int) this.srcBuf[this.bytesRead++];
-		}
-
-
-		@Override
-		public int read(final byte[] buf) throws IOException {
-			return read(buf, 0, buf.length);
-		}
-
-		@Override
-		public int read(final byte[] buf, final int off, final int len) throws IOException {
-			if (this.length - this.bytesRead == 0) {
-				return -1;
-			}
-
-			final int bytesToCopy = Math.min(len, this.length - this.bytesRead);
-			System.arraycopy(srcBuf, this.bytesRead, buf, off, bytesToCopy);
-			this.bytesRead += bytesToCopy;
-
-			return bytesToCopy;
-		}
-
-		@Override
-		public int available() throws IOException {
-
-			return (this.length - bytesRead);
-		}
-
-		@Override
-		public long skip(final long n) throws IOException {
-
-			int bytesToSkip = (int) Math.min(n, Integer.MAX_VALUE);
-			bytesToSkip = Math.min(this.length - this.bytesRead, bytesToSkip);
-
-			this.bytesRead += bytesToSkip;
-
-			return bytesToSkip;
-		}
-	}
-
-	S3DataOutputStream(final AmazonS3Client s3Client, final String bucket, final String object, final byte[] buf,
-			final boolean useRRS) {
-
-		this.s3Client = s3Client;
-		this.bucket = bucket;
-		this.object = object;
-		this.buf = buf;
-		this.useRRS = useRRS;
-	}
-
-
-	@Override
-	public void write(final int b) throws IOException {
-		// Upload buffer to S3
-		if (this.bytesWritten == this.buf.length) {
-			uploadPartAndFlushBuffer();
-		}
-
-		this.buf[this.bytesWritten++] = (byte) b;
-	}
-
-
-	@Override
-	public void write(final byte[] b, final int off, final int len) throws IOException {
-		int nextPos = off;
-
-		while (nextPos < len) {
-			// Upload buffer to S3
-			if (this.bytesWritten == this.buf.length) {
-				uploadPartAndFlushBuffer();
-			}
-
-			final int bytesToCopy = Math.min(this.buf.length - this.bytesWritten, len - nextPos);
-			System.arraycopy(b, nextPos, this.buf, this.bytesWritten, bytesToCopy);
-			this.bytesWritten += bytesToCopy;
-			nextPos += bytesToCopy;
-		}
-	}
-
-
-	@Override
-	public void write(final byte[] b) throws IOException {
-		write(b, 0, b.length);
-	}
-
-
-	@Override
-	public void close() throws IOException {
-		if (this.uploadId == null) {
-			// This is not a multipart upload
-
-			// No data has been written
-			if (this.bytesWritten == 0) {
-				return;
-			}
-
-			final InputStream is = new InternalUploadInputStream(this.buf, this.bytesWritten);
-			final ObjectMetadata om = new ObjectMetadata();
-			om.setContentLength(this.bytesWritten);
-
-			final PutObjectRequest por = new PutObjectRequest(this.bucket, this.object, is, om);
-			if (this.useRRS) {
-				por.setStorageClass(StorageClass.ReducedRedundancy);
-			} else {
-				por.setStorageClass(StorageClass.Standard);
-			}
-
-			try {
-				this.s3Client.putObject(por);
-			} catch (AmazonServiceException e) {
-				throw new IOException(StringUtils.stringifyException(e));
-			}
-
-			this.bytesWritten = 0;
-
-		} else {
-
-			if (this.bytesWritten > 0) {
-				uploadPartAndFlushBuffer();
-			}
-
-			boolean operationSuccessful = false;
-			try {
-				final CompleteMultipartUploadRequest request = new CompleteMultipartUploadRequest(this.bucket,
-					this.object,
-					this.uploadId, this.partETags);
-				this.s3Client.completeMultipartUpload(request);
-
-				operationSuccessful = true;
-
-			} catch (AmazonServiceException e) {
-				throw new IOException(StringUtils.stringifyException(e));
-			} finally {
-				if (!operationSuccessful) {
-					abortUpload();
-				}
-			}
-		}
-	}
-
-	@Override
-	public void sync() throws IOException {
-		// can do nothing here
-	}
-	
-	@Override
-	public void flush() throws IOException {
-		// Flush does nothing in this implementation since we ways have to transfer at least 5 MB in a multipart upload
-	}
-
-	private void uploadPartAndFlushBuffer() throws IOException {
-
-		boolean operationSuccessful = false;
-
-		if (this.uploadId == null) {
-			this.uploadId = initiateMultipartUpload();
-		}
-
-		try {
-			if (this.partNumber >= MAX_PART_NUMBER) {
-				throw new IOException("Cannot upload any more data: maximum part number reached");
-			}
-
-			final InputStream inputStream = new InternalUploadInputStream(this.buf, this.bytesWritten);
-			final UploadPartRequest request = new UploadPartRequest();
-			request.setBucketName(this.bucket);
-			request.setKey(this.object);
-			request.setInputStream(inputStream);
-			request.setUploadId(this.uploadId);
-			request.setPartSize(this.bytesWritten);
-			request.setPartNumber(this.partNumber++);
-
-			final UploadPartResult result = this.s3Client.uploadPart(request);
-			this.partETags.add(result.getPartETag());
-
-			this.bytesWritten = 0;
-			operationSuccessful = true;
-
-		}
-		catch (AmazonServiceException e) {
-			throw new IOException(e.getMessage(), e);
-		}
-		finally {
-			if (!operationSuccessful) {
-				abortUpload();
-			}
-		}
-	}
-
-	private String initiateMultipartUpload() throws IOException {
-
-		boolean operationSuccessful = false;
-		final InitiateMultipartUploadRequest request = new InitiateMultipartUploadRequest(this.bucket, this.object);
-		if (this.useRRS) {
-			request.setStorageClass(StorageClass.ReducedRedundancy);
-		} else {
-			request.setStorageClass(StorageClass.Standard);
-		}
-
-		try {
-
-			final InitiateMultipartUploadResult result = this.s3Client.initiateMultipartUpload(request);
-			operationSuccessful = true;
-			return result.getUploadId();
-
-		}
-		catch (AmazonServiceException e) {
-			throw new IOException(e.getMessage(), e);
-		}
-		finally {
-			if (!operationSuccessful) {
-				abortUpload();
-			}
-		}
-	}
-
-	private void abortUpload() {
-		if (this.uploadId == null) {
-			// This is not a multipart upload, nothing to do here
-			return;
-		}
-
-		try {
-			final AbortMultipartUploadRequest request = new AbortMultipartUploadRequest(this.bucket, this.object,
-				this.uploadId);
-			this.s3Client.abortMultipartUpload(request);
-		}
-		catch (AmazonServiceException e) {
-			// Ignore exception
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DirectoryStructure.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DirectoryStructure.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DirectoryStructure.java
deleted file mode 100644
index aaf5284..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DirectoryStructure.java
+++ /dev/null
@@ -1,87 +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.runtime.fs.s3;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.core.fs.Path;
-
-public final class S3DirectoryStructure {
-
-	private final String basePath;
-
-	private final Map<Path, S3BucketObjectPair> cache = new HashMap<Path, S3BucketObjectPair>();
-
-	S3DirectoryStructure(final String basePath) {
-		this.basePath = basePath;
-	}
-
-	S3BucketObjectPair toBucketObjectPair(final Path path) {
-
-		S3BucketObjectPair bop = this.cache.get(path);
-		if (bop != null) {
-			return bop;
-		}
-
-		final URI uri = path.toUri();
-		String p = uri.getPath();
-		if (!this.basePath.isEmpty() && !p.contains(this.basePath)) {
-			throw new IllegalArgumentException(path + " is not a valid path for the file system");
-		}
-
-		// Extract the base path
-		if (!this.basePath.isEmpty()) {
-			final int pos = p.indexOf(this.basePath);
-			p = p.substring(pos + this.basePath.length());
-		}
-
-		// Remove leading SEPARATOR
-		if (!p.isEmpty()) {
-			if (p.charAt(0) == Path.SEPARATOR_CHAR) {
-				p = p.substring(1);
-			}
-		}
-
-		if (p.isEmpty()) {
-			bop = new S3BucketObjectPair(null, null);
-			this.cache.put(path, bop);
-			return bop;
-		}
-
-		final int objectPos = p.indexOf(Path.SEPARATOR_CHAR);
-		if (objectPos < 0) {
-			bop = new S3BucketObjectPair(p, null);
-		} else {
-			final String bucket = p.substring(0, objectPos);
-			final String object = p.substring(objectPos + 1);
-			if (object.isEmpty()) {
-				bop = new S3BucketObjectPair(bucket, null);
-			} else {
-				bop = new S3BucketObjectPair(bucket, object);
-			}
-		}
-
-		this.cache.put(path, bop);
-
-		return bop;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileStatus.java
deleted file mode 100644
index bb0daff..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileStatus.java
+++ /dev/null
@@ -1,95 +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.runtime.fs.s3;
-
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.Path;
-
-public final class S3FileStatus implements FileStatus {
-
-	private final Path path;
-
-	private final long length;
-
-	private final boolean isDir;
-
-	private final long modificationTime;
-	
-	private final long accessTime;
-	
-	S3FileStatus(final Path path, final long length, final boolean isDir, final long modificationTime,
-			final long accessTime) {
-		this.path = path;
-		this.length = length;
-		this.isDir = isDir;
-		this.modificationTime = modificationTime;
-		this.accessTime = accessTime;
-	}
-
-
-	@Override
-	public long getLen() {
-
-		return this.length;
-	}
-
-
-	@Override
-	public long getBlockSize() {
-
-		return this.length;
-	}
-
-
-	@Override
-	public short getReplication() {
-
-		return 1;
-	}
-
-
-	@Override
-	public long getModificationTime() {
-		
-		return this.modificationTime;
-	}
-
-
-	@Override
-	public long getAccessTime() {
-		
-		return this.accessTime;
-	}
-
-
-	@Override
-	public boolean isDir() {
-
-		return this.isDir;
-	}
-
-
-	@Override
-	public Path getPath() {
-
-		return this.path;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileSystem.java
deleted file mode 100644
index 522d90e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3FileSystem.java
+++ /dev/null
@@ -1,786 +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.runtime.fs.s3;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.StringUtils;
-
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.model.Bucket;
-import com.amazonaws.services.s3.model.ObjectListing;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.Owner;
-import com.amazonaws.services.s3.model.S3ObjectSummary;
-
-/**
- * This class provides a {@link FileSystem} implementation which is backed by Amazon's Simple Storage Service (S3). The
- * implementation uses the REST API of Amazon S3 to facilitate the communication and read/write the data.
- * 
- */
-public final class S3FileSystem extends FileSystem {
-
-	/**
-	 * The logging object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(S3FileSystem.class);
-
-	/**
-	 * The configuration key to access the S3 host.
-	 */
-	public static final String S3_HOST_KEY = "fs.s3.host";
-
-	/**
-	 * The configuration key to access the S3 port.
-	 */
-	public static final String S3_PORT_KEY = "fs.s3.port";
-
-	/**
-	 * The configuration key to access the S3 Reduced Redundancy Storage setting.
-	 */
-	public static final String S3_RRS_KEY = "fs.s3.rrs";
-
-	/**
-	 * The configuration key to access the S3 access key.
-	 */
-	public static final String S3_ACCESS_KEY_KEY = "fs.s3.accessKey";
-
-	/**
-	 * The configuration key to access the S3 secret key.
-	 */
-	public static final String S3_SECRET_KEY_KEY = "fs.s3.secretKey";
-
-	/**
-	 * The default host to connect to.
-	 */
-	private static final String DEFAULT_S3_HOST = "s3.amazonaws.com";
-
-	/**
-	 * The default setting whether to use S3 Reduced Redundancy Storage
-	 */
-	private static final boolean DEFAULT_S3_RRS = true;
-
-	/**
-	 * The default port to connect to.
-	 */
-	private static final int DEFAULT_S3_PORT = 80;
-
-	/**
-	 * The prefix of the HTTP protocol.
-	 */
-	private static final String HTTP_PREFIX = "http";
-
-	/**
-	 * The error code for "resource not found" according to the HTTP protocol.
-	 */
-	private static final int HTTP_RESOURCE_NOT_FOUND_CODE = 404;
-
-	/**
-	 * The character which S3 uses internally to indicate an object represents a directory.
-	 */
-	private static final char S3_DIRECTORY_SEPARATOR = '/';
-
-	/**
-	 * The scheme which is used by this file system.
-	 */
-	public static final String S3_SCHEME = "s3";
-
-	/**
-	 * The character set with which the URL is expected to be encoded
-	 */
-	private static final String URL_ENCODE_CHARACTER = "UTF-8";
-
-	/**
-	 * The host to address the REST requests to.
-	 */
-	private String host = null;
-
-	private int port = -1;
-
-	private URI s3Uri = null;
-
-	private AmazonS3Client s3Client = null;
-
-	private S3DirectoryStructure directoryStructure = null;
-
-	private final boolean useRRS;
-
-	public S3FileSystem() {
-
-		this.useRRS = GlobalConfiguration.getBoolean(S3_RRS_KEY, DEFAULT_S3_RRS);
-		LOG.info("Creating new S3 file system binding with Reduced Redundancy Storage "
-			+ (this.useRRS ? "enabled" : "disabled"));
-	}
-
-
-	@Override
-	public Path getWorkingDirectory() {
-
-		return new Path(this.s3Uri);
-	}
-
-	public Path getHomeDirectory() {
-		return new Path(this.s3Uri);
-	}
-
-	@Override
-	public URI getUri() {
-
-		return this.s3Uri;
-	}
-
-
-	@Override
-	public void initialize(URI name) throws IOException {
-
-		this.host = name.getHost();
-		if (this.host == null) {
-			LOG.debug("Provided URI does not provide a host to connect to, using configuration...");
-			this.host = GlobalConfiguration.getString(S3_HOST_KEY, DEFAULT_S3_HOST);
-		}
-
-		this.port = name.getPort();
-		if (this.port == -1) {
-			LOG.debug("Provided URI does not provide a port to connect to, using configuration...");
-			this.port = GlobalConfiguration.getInteger(S3_PORT_KEY, DEFAULT_S3_PORT);
-		}
-
-		final String userInfo = name.getUserInfo();
-
-		String awsAccessKey = null;
-		String awsSecretKey = null;
-
-		if (userInfo != null) {
-
-			final String[] splits = userInfo.split(":");
-			if (splits.length > 1) {
-				awsAccessKey = URLDecoder.decode(splits[0], URL_ENCODE_CHARACTER);
-				awsSecretKey = URLDecoder.decode(splits[1], URL_ENCODE_CHARACTER);
-			}
-		}
-
-		if (awsAccessKey == null) {
-			LOG.debug("Provided URI does not provide an access key to Amazon S3, using configuration...");
-			awsAccessKey = GlobalConfiguration.getString(S3_ACCESS_KEY_KEY, null);
-			if (awsAccessKey == null) {
-				throw new IOException("Cannot determine access key to Amazon S3. Please make " +
-						"sure to configure it by setting the configuration key '"
-						+ S3_ACCESS_KEY_KEY + "'.");
-			}
-		}
-
-		if (awsSecretKey == null) {
-			LOG.debug("Provided URI does not provide a secret key to Amazon S3, using configuration...");
-			awsSecretKey = GlobalConfiguration.getString(S3_SECRET_KEY_KEY, null);
-			if (awsSecretKey == null) {
-				throw new IOException("Cannot determine secret key to Amazon S3. Please make " +
-						"sure to configure it by setting the configuration key '"
-						+ S3_SECRET_KEY_KEY + "'.");
-			}
-		}
-
-		final AWSCredentials credentials = new BasicAWSCredentials(awsAccessKey, awsSecretKey);
-		this.s3Client = new AmazonS3Client(credentials);
-
-		initializeDirectoryStructure(name);
-	}
-
-	private void initializeDirectoryStructure(final URI name) throws IOException {
-
-		String basePath = name.getPath();
-		while (true) {
-
-			try {
-				final String endpoint = new URL(HTTP_PREFIX, this.host, this.port, basePath).toString();
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Trying S3 endpoint " + endpoint);
-				}
-
-				this.s3Client.setEndpoint(endpoint);
-				final Owner owner = this.s3Client.getS3AccountOwner();
-				LOG.info("Successfully established connection to Amazon S3 using the endpoint " + endpoint);
-				LOG.info("Amazon S3 user is " + owner.getDisplayName());
-
-				break;
-			} catch (MalformedURLException e) {
-				throw new IOException(StringUtils.stringifyException(e));
-			} catch (AmazonClientException e) {
-
-				// Truncate path
-				if (basePath.isEmpty()) {
-					throw new IOException("Cannot establish connection to Amazon S3: "
-						+ StringUtils.stringifyException(e));
-				} else {
-					final int pos = basePath.lastIndexOf(Path.SEPARATOR);
-					if (pos < 0) {
-						basePath = "";
-					} else {
-						basePath = basePath.substring(0, pos);
-					}
-				}
-			}
-		}
-
-		// Set the S3 URI
-		try {
-			this.s3Uri = new URI(S3_SCHEME, (String) null, this.host, this.port, basePath, null, null);
-		} catch (URISyntaxException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-
-		// Finally, create directory structure object
-		this.directoryStructure = new S3DirectoryStructure(basePath);
-	}
-
-
-	@Override
-	public FileStatus getFileStatus(final Path f) throws IOException {
-
-		final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-
-		// This is the S3:/// base directory
-		if (!bop.hasBucket() && !bop.hasObject()) {
-			return new S3FileStatus(f, 0L, true, 0L, 0L);
-		}
-
-		try {
-			if (bop.hasBucket() && !bop.hasObject()) {
-
-				final List<Bucket> buckets = this.s3Client.listBuckets();
-				final Iterator<Bucket> it = buckets.iterator();
-
-				// Iterator throw list of buckets to find out creation date
-				while (it.hasNext()) {
-
-					final Bucket bucket = it.next();
-					if (bop.getBucket().equals(bucket.getName())) {
-
-						final long creationDate = dateToLong(bucket.getCreationDate());
-						// S3 does not track access times, so this implementation always sets it to 0
-						return new S3FileStatus(f, 0L, true, creationDate, 0L);
-					}
-				}
-
-				throw new FileNotFoundException("Cannot find " + f.toUri());
-			}
-
-			try {
-				final ObjectMetadata om = this.s3Client.getObjectMetadata(bop.getBucket(), bop.getObject());
-				final long modificationDate = dateToLong(om.getLastModified());
-				// S3 does not track access times, so this implementation always sets it to 0
-				if (objectRepresentsDirectory(bop.getObject(), om.getContentLength())) {
-					return new S3FileStatus(f, 0L, true, modificationDate, 0L);
-				} else {
-					return new S3FileStatus(f, om.getContentLength(), false, modificationDate, 0L);
-				}
-
-			} catch (AmazonServiceException e) {
-				if (e.getStatusCode() == HTTP_RESOURCE_NOT_FOUND_CODE) {
-					throw new FileNotFoundException("Cannot find " + f.toUri());
-				} else {
-					throw e;
-				}
-			}
-		} catch (AmazonClientException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-	}
-
-	private static long dateToLong(final Date date) {
-
-		if (date == null) {
-			return 0L;
-		}
-
-		return date.getTime();
-	}
-
-
-	@Override
-	public BlockLocation[] getFileBlockLocations(final FileStatus file, final long start, final long len)
-			throws IOException {
-
-		if ((start + len) > file.getLen()) {
-			return null;
-		}
-
-		final S3BlockLocation bl = new S3BlockLocation(this.host, file.getLen());
-
-		return new BlockLocation[] { bl };
-	}
-
-
-	@Override
-	public FSDataInputStream open(final Path f, final int bufferSize) throws IOException {
-
-		return open(f); // Ignore bufferSize
-	}
-
-
-	@Override
-	public FSDataInputStream open(final Path f) throws IOException {
-
-		final FileStatus fileStatus = getFileStatus(f); // Will throw FileNotFoundException if f does not exist
-
-		// Make sure f is not a directory
-		if (fileStatus.isDir()) {
-			throw new IOException("Cannot open " + f.toUri() + " because it is a directory");
-		}
-
-		final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-		if (!bop.hasBucket() || !bop.hasObject()) {
-			throw new IOException(f.toUri() + " cannot be opened");
-		}
-
-		return new S3DataInputStream(this.s3Client, bop.getBucket(), bop.getObject());
-	}
-
-
-	@Override
-	public FileStatus[] listStatus(final Path f) throws IOException {
-
-		final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-
-		try {
-
-			if (!bop.hasBucket()) {
-
-				final List<Bucket> list = this.s3Client.listBuckets();
-				final S3FileStatus[] array = new S3FileStatus[list.size()];
-				final Iterator<Bucket> it = list.iterator();
-				int i = 0;
-				while (it.hasNext()) {
-					final Bucket bucket = it.next();
-					final long creationDate = dateToLong(bucket.getCreationDate());
-					// S3 does not track access times, so this implementation always sets it to 0
-					final S3FileStatus status = new S3FileStatus(extendPath(f, bucket.getName()
-						+ S3_DIRECTORY_SEPARATOR), 0, true, creationDate, 0L);
-					array[i++] = status;
-				}
-
-				return array;
-			}
-
-			if (bop.hasBucket() && !bop.hasObject()) {
-
-				// Check if the bucket really exists
-				if (!this.s3Client.doesBucketExist(bop.getBucket())) {
-					throw new FileNotFoundException("Cannot find " + f.toUri());
-				}
-
-				return listBucketContent(f, bop);
-
-			} else {
-
-				final ObjectMetadata omd = this.s3Client.getObjectMetadata(bop.getBucket(), bop.getObject());
-				if (objectRepresentsDirectory(bop.getObject(), omd.getContentLength())) {
-
-					return listBucketContent(f, bop);
-
-				} else {
-					final S3FileStatus fileStatus = new S3FileStatus(f, omd.getContentLength(), false,
-						dateToLong(omd.getLastModified()), 0L);
-
-					return new FileStatus[] { fileStatus };
-				}
-
-			}
-
-		} catch (AmazonClientException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-	}
-
-	private S3FileStatus[] listBucketContent(final Path f, final S3BucketObjectPair bop) throws IOException {
-
-		ObjectListing listing = null;
-		final List<S3FileStatus> resultList = new ArrayList<S3FileStatus>();
-
-		final int depth = (bop.hasObject() ? getDepth(bop.getObject()) + 1 : 0);
-
-		while (true) {
-
-			if (listing == null) {
-				if (bop.hasObject()) {
-					listing = this.s3Client.listObjects(bop.getBucket(), bop.getObject());
-				} else {
-					listing = this.s3Client.listObjects(bop.getBucket());
-				}
-			} else {
-				listing = this.s3Client.listNextBatchOfObjects(listing);
-			}
-
-			final List<S3ObjectSummary> list = listing.getObjectSummaries();
-			final Iterator<S3ObjectSummary> it = list.iterator();
-			while (it.hasNext()) {
-
-				final S3ObjectSummary os = it.next();
-				String key = os.getKey();
-
-				final int childDepth = getDepth(os.getKey());
-
-				if (childDepth != depth) {
-					continue;
-				}
-
-				// Remove the prefix
-				if (bop.hasObject()) {
-					if (key.startsWith(bop.getObject())) {
-						key = key.substring(bop.getObject().length());
-					}
-
-					// This has been the prefix itself
-					if (key.isEmpty()) {
-						continue;
-					}
-				}
-
-				final long modificationDate = dateToLong(os.getLastModified());
-
-				S3FileStatus fileStatus;
-				if (objectRepresentsDirectory(os)) {
-					fileStatus = new S3FileStatus(extendPath(f, key), 0, true, modificationDate, 0L);
-				} else {
-					fileStatus = new S3FileStatus(extendPath(f, key), os.getSize(), false, modificationDate, 0L);
-				}
-
-				resultList.add(fileStatus);
-			}
-
-			if (!listing.isTruncated()) {
-				break;
-			}
-		}
-
-		/*
-		 * System.out.println("---- RETURN CONTENT ----");
-		 * for (final FileStatus entry : resultList) {
-		 * System.out.println(entry.getPath());
-		 * }
-		 * System.out.println("------------------------");
-		 */
-
-		return resultList.toArray(new S3FileStatus[0]);
-
-	}
-
-	private static int getDepth(final String key) {
-
-		int depth = 0;
-		int nextStartPos = 0;
-
-		final int length = key.length();
-
-		while (nextStartPos < length) {
-
-			final int sepPos = key.indexOf(S3_DIRECTORY_SEPARATOR, nextStartPos);
-			if (sepPos < 0) {
-				break;
-			} else {
-				++depth;
-				nextStartPos = sepPos + 1;
-			}
-		}
-
-		if (length > 0) {
-			if (key.charAt(length - 1) == S3_DIRECTORY_SEPARATOR) {
-				--depth;
-			}
-		}
-
-		return depth;
-	}
-
-
-	@Override
-	public boolean delete(Path f, boolean recursive) throws IOException {
-
-		try {
-			final FileStatus fileStatus = getFileStatus(f); // Will throw a FileNotFoundException if f is invalid
-			final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-
-			if (fileStatus.isDir()) {
-
-				boolean retVal = false;
-				final FileStatus[] dirContent = listStatus(f);
-				if (dirContent.length > 0) {
-					// Directory is not empty
-					if (!recursive) {
-						throw new IOException("Found non-empty directory " + f
-							+ " while performing non-recursive delete");
-					}
-
-					for (final FileStatus entry : dirContent) {
-
-						if (delete(entry.getPath(), true)) {
-							retVal = true;
-						}
-					}
-				}
-
-				// Now the directory is empty
-
-				if (!bop.hasBucket()) {
-					// This is the root directory, do not delete this
-					return retVal;
-				}
-
-				if (!bop.hasObject()) {
-					// This is a real bucket
-					this.s3Client.deleteBucket(bop.getBucket());
-				} else {
-					// This directory is actually represented by an object in S3
-					this.s3Client.deleteObject(bop.getBucket(), bop.getObject());
-				}
-			} else {
-				// This is a file
-				this.s3Client.deleteObject(bop.getBucket(), bop.getObject());
-			}
-		} catch (AmazonClientException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public boolean mkdirs(final Path f) throws IOException {
-
-		final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-		if (!bop.hasBucket() && !bop.hasObject()) {
-			// Ignore this call
-			return false;
-		}
-
-		boolean retCode = false;
-
-		try {
-
-			// Make sure the bucket exists
-			if (bop.hasBucket()) {
-				if (this.s3Client.doesBucketExist(bop.getBucket())) {
-				} else {
-					this.s3Client.createBucket(bop.getBucket());
-					retCode = true;
-				}
-			}
-
-			if (bop.hasObject()) {
-
-				// Make sure object name ends with a directory separator character
-				String object = bop.getObject();
-				if (!object.isEmpty()) {
-					if (object.charAt(object.length() - 1) != S3_DIRECTORY_SEPARATOR) {
-						object = object.concat(Character.toString(S3_DIRECTORY_SEPARATOR));
-					}
-				}
-
-				while (true) {
-
-					try {
-						this.s3Client.getObjectMetadata(bop.getBucket(), object);
-					} catch (AmazonServiceException e) {
-						if (e.getStatusCode() == HTTP_RESOURCE_NOT_FOUND_CODE) {
-							createEmptyObject(bop.getBucket(), object);
-
-							if (object.length() > 1) {
-								final int nextPos = object.lastIndexOf(S3_DIRECTORY_SEPARATOR, object.length() - 2);
-								if (nextPos >= 0) {
-									object = object.substring(0, nextPos + 1);
-									continue;
-								}
-							}
-
-						} else {
-							// Rethrow the exception
-							throw e;
-						}
-					}
-
-					// Object already exists, exit
-					break;
-				}
-			}
-		} catch (AmazonClientException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-
-		return retCode;
-	}
-
-	private void createEmptyObject(final String bucketName, final String objectName) {
-
-		final InputStream im = new InputStream() {
-
-			@Override
-			public int read() throws IOException {
-
-				return -1;
-			}
-		};
-
-		final ObjectMetadata om = new ObjectMetadata();
-		om.setContentLength(0L);
-
-		this.s3Client.putObject(bucketName, objectName, im, om);
-	}
-
-
-	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
-			final short replication, final long blockSize)
-			throws IOException {
-
-		if (!overwrite && exists(f)) {
-			throw new IOException(f.toUri() + " already exists");
-		}
-
-		final S3BucketObjectPair bop = this.directoryStructure.toBucketObjectPair(f);
-		if (!bop.hasBucket() || !bop.hasObject()) {
-			throw new IOException(f.toUri() + " is not a valid path to create a new file");
-		}
-
-		if (bufferSize < S3DataOutputStream.MINIMUM_MULTIPART_SIZE) {
-			throw new IOException("Provided buffer must be at least " + S3DataOutputStream.MINIMUM_MULTIPART_SIZE
-				+ " bytes");
-		}
-
-		final byte[] buf = new byte[bufferSize]; // TODO: Use memory manager to allocate larger pages
-
-		return new S3DataOutputStream(this.s3Client, bop.getBucket(), bop.getObject(), buf, this.useRRS);
-	}
-
-
-	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
-
-		return create(f, overwrite, S3DataOutputStream.MINIMUM_MULTIPART_SIZE, (short) 1, 1024L);
-	}
-
-	private boolean objectRepresentsDirectory(final S3ObjectSummary os) {
-
-		return objectRepresentsDirectory(os.getKey(), os.getSize());
-	}
-
-	private boolean objectRepresentsDirectory(final String name, final long size) {
-
-		if (name.isEmpty()) {
-			return false;
-		}
-
-		if (name.charAt(name.length() - 1) == S3_DIRECTORY_SEPARATOR && size == 0L) {
-			return true;
-		}
-
-		return false;
-	}
-
-	static Path extendPath(final Path parent, final String extension) throws IOException {
-
-		final URI parentUri = parent.toUri();
-
-		if (extension.isEmpty()) {
-			return parent;
-		}
-
-		final String path = parentUri.getPath();
-		String extendedPath;
-		if (path.isEmpty()) {
-			if (extension.charAt(0) == Path.SEPARATOR_CHAR) {
-				extendedPath = extension;
-			} else {
-				extendedPath = Path.SEPARATOR + extension;
-			}
-		} else {
-			if (path.charAt(path.length() - 1) == Path.SEPARATOR_CHAR) {
-				if (extension.charAt(0) == Path.SEPARATOR_CHAR) {
-					if (extension.length() > 1) {
-						extendedPath = path + extension.substring(1);
-					} else {
-						extendedPath = path;
-					}
-				} else {
-					extendedPath = path + extension;
-				}
-			} else {
-				if (extension.charAt(0) == Path.SEPARATOR_CHAR) {
-					extendedPath = path + extension;
-				} else {
-					extendedPath = path + Path.SEPARATOR + extension;
-				}
-			}
-		}
-
-		try {
-			final URI extendedUri = new URI(parentUri.getScheme(),
-				((parentUri.getAuthority() != null) ? parentUri.getAuthority() : ""), extendedPath,
-				parentUri.getQuery(), parentUri.getFragment());
-			return new Path(extendedUri);
-		} catch (URISyntaxException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		}
-	}
-
-
-	@Override
-	public boolean rename(final Path src, final Path dst) throws IOException {
-
-		throw new UnsupportedOperationException("This method is not yet implemented");
-	}
-
-	@Override
-	public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean createDirectory) throws IOException {
-		if (createDirectory) {
-			// make sure that the path is terminated with a slash, S3 is very particular about this
-			outPath = outPath.suffix("/");
-		}
-		return super.initOutPathDistFS(outPath, writeMode, createDirectory);
-	}
-	
-	@Override
-	public boolean isDistributedFS() {
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9005779/flink-runtime/src/test/java/org/apache/flink/runtime/fs/s3/S3FileSystemTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/fs/s3/S3FileSystemTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/fs/s3/S3FileSystemTest.java
deleted file mode 100644
index b8cd99d..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/fs/s3/S3FileSystemTest.java
+++ /dev/null
@@ -1,465 +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.runtime.fs.s3;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This test checks the S3 implementation of the {@link FileSystem} interface.
- * 
- */
-public class S3FileSystemTest {
-
-	/**
-	 * The length of the bucket/object names used in this test.
-	 */
-	private static final int NAME_LENGTH = 32;
-
-	/**
-	 * The alphabet to generate the random bucket/object names from.
-	 */
-	private static final char[] ALPHABET = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o',
-		'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
-
-	/**
-	 * The size of the byte buffer used during the tests in bytes.
-	 */
-	private static final int TEST_BUFFER_SIZE = 128;
-
-	/**
-	 * The size of the small test file in bytes.
-	 */
-	private static final int SMALL_FILE_SIZE = 512;
-
-	/**
-	 * The size of the large test file in bytes.
-	 */
-	private static final int LARGE_FILE_SIZE = 1024 * 1024 * 12; // 12 MB
-
-	/**
-	 * The modulus to be used when generating the test data. Must not be larger than 128.
-	 */
-	private static final int MODULUS = 128;
-
-	private static final String S3_BASE_URI = "s3:///";
-
-	/**
-	 * Tries to read the AWS access key and the AWS secret key from the environments variables. If accessing these keys
-	 * fails, all tests will be skipped and marked as successful.
-	 */
-	@Before
-	public void initKeys() {
-		final String accessKey = System.getenv("AK");
-		final String secretKey = System.getenv("SK");
-		
-		if (accessKey != null || secretKey != null) {
-			Configuration conf = new Configuration();
-			if (accessKey != null) {
-				conf.setString(S3FileSystem.S3_ACCESS_KEY_KEY, accessKey);
-			}
-			if (secretKey != null) {
-				conf.setString(S3FileSystem.S3_SECRET_KEY_KEY, secretKey);
-			}
-			GlobalConfiguration.includeConfiguration(conf);
-		}
-	}
-
-	/**
-	 * This test creates and deletes a bucket inside S3 and checks it is correctly displayed inside the directory
-	 * listing.
-	 */
-	@Test
-	public void createAndDeleteBucketTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String bucketName = getRandomName();
-		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
-
-		try {
-
-			final FileSystem fs = bucketPath.getFileSystem();
-
-			// Create directory
-			fs.mkdirs(bucketPath);
-
-			// Check if directory is correctly displayed in file system hierarchy
-			final FileStatus[] content = fs.listStatus(new Path(S3_BASE_URI));
-			boolean entryFound = false;
-			for (final FileStatus entry : content) {
-				if (bucketPath.equals(entry.getPath())) {
-					entryFound = true;
-					break;
-				}
-			}
-
-			if (!entryFound) {
-				fail("Cannot find entry " + bucketName + " in directory " + S3_BASE_URI);
-			}
-
-			// Check the concrete directory file status
-			try {
-				final FileStatus directoryFileStatus = fs.getFileStatus(bucketPath);
-				assertTrue(directoryFileStatus.isDir());
-				assertEquals(0L, directoryFileStatus.getAccessTime());
-				assertTrue(directoryFileStatus.getModificationTime() > 0L);
-
-			} catch (FileNotFoundException e) {
-				fail(e.getMessage());
-			}
-
-			// Delete the bucket
-			fs.delete(bucketPath, true);
-
-			// Make sure the bucket no longer exists
-			try {
-				fs.getFileStatus(bucketPath);
-				fail("Expected FileNotFoundException for " + bucketPath.toUri());
-			} catch (FileNotFoundException e) {
-				// This is an expected exception
-			}
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads the a larger test file in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 */
-	@Test
-	public void createAndReadLargeFileTest() {
-
-		try {
-			createAndReadFileTest(LARGE_FILE_SIZE);
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads the a small test file in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 */
-	@Test
-	public void createAndReadSmallFileTest() {
-
-		try {
-			createAndReadFileTest(SMALL_FILE_SIZE);
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * The tests checks the mapping of the file system directory structure to the underlying bucket/object model of
-	 * Amazon S3.
-	 */
-	@Test
-	public void multiLevelDirectoryTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String dirName = getRandomName();
-		final String subdirName = getRandomName();
-		final String subsubdirName = getRandomName();
-		final String fileName = getRandomName();
-		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
-		final Path subdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR);
-		final Path subsubdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR
-			+ subsubdirName + Path.SEPARATOR);
-		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR + fileName);
-
-		try {
-
-			final FileSystem fs = dir.getFileSystem();
-
-			fs.mkdirs(subsubdir);
-
-			final OutputStream os = fs.create(file, true);
-			generateTestData(os, SMALL_FILE_SIZE);
-			os.close();
-
-			// On this directory levels there should only be one subdirectory
-			FileStatus[] list = fs.listStatus(dir);
-			int numberOfDirs = 0;
-			int numberOfFiles = 0;
-			for (final FileStatus entry : list) {
-
-				if (entry.isDir()) {
-					++numberOfDirs;
-					assertEquals(subdir, entry.getPath());
-				} else {
-					fail(entry.getPath() + " is a file which must not appear on this directory level");
-				}
-			}
-
-			assertEquals(1, numberOfDirs);
-			assertEquals(0, numberOfFiles);
-
-			list = fs.listStatus(subdir);
-			numberOfDirs = 0;
-
-			for (final FileStatus entry : list) {
-				if (entry.isDir()) {
-					assertEquals(subsubdir, entry.getPath());
-					++numberOfDirs;
-				} else {
-					assertEquals(file, entry.getPath());
-					++numberOfFiles;
-				}
-			}
-
-			assertEquals(1, numberOfDirs);
-			assertEquals(1, numberOfFiles);
-
-			fs.delete(dir, true);
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * This test checks the S3 implementation of the file system method to retrieve the block locations of a file.
-	 */
-	@Test
-	public void blockLocationTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String dirName = getRandomName();
-		final String fileName = getRandomName();
-		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
-		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + fileName);
-
-		try {
-
-			final FileSystem fs = dir.getFileSystem();
-
-			fs.mkdirs(dir);
-
-			final OutputStream os = fs.create(file, true);
-			generateTestData(os, SMALL_FILE_SIZE);
-			os.close();
-
-			final FileStatus fileStatus = fs.getFileStatus(file);
-			assertNotNull(fileStatus);
-
-			BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE + 1);
-			assertNull(blockLocations);
-
-			blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE);
-			assertEquals(1, blockLocations.length);
-
-			final BlockLocation bl = blockLocations[0];
-			assertNotNull(bl.getHosts());
-			assertEquals(1, bl.getHosts().length);
-			assertEquals(SMALL_FILE_SIZE, bl.getLength());
-			assertEquals(0, bl.getOffset());
-			final URI s3Uri = fs.getUri();
-			assertNotNull(s3Uri);
-			assertEquals(s3Uri.getHost(), bl.getHosts()[0]);
-
-			fs.delete(dir, true);
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads a file with the given size in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 * 
-	 * @param fileSize
-	 *        the size of the file to be generated in bytes
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while writing or reading the test file
-	 */
-	private void createAndReadFileTest(final int fileSize) throws IOException {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String bucketName = getRandomName();
-		final String objectName = getRandomName();
-		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
-		final Path objectPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR + objectName);
-
-		FileSystem fs = bucketPath.getFileSystem();
-
-		// Create test bucket
-		fs.mkdirs(bucketPath);
-
-		// Write test file to S3
-		final FSDataOutputStream outputStream = fs.create(objectPath, false);
-		generateTestData(outputStream, fileSize);
-		outputStream.close();
-
-		// Now read the same file back from S3
-		final FSDataInputStream inputStream = fs.open(objectPath);
-		testReceivedData(inputStream, fileSize);
-		inputStream.close();
-
-		// Delete test bucket
-		fs.delete(bucketPath, true);
-	}
-
-	/**
-	 * Receives test data from the given input stream and checks the size of the data as well as the pattern inside the
-	 * received data.
-	 * 
-	 * @param inputStream
-	 *        the input stream to read the test data from
-	 * @param expectedSize
-	 *        the expected size of the data to be read from the input stream in bytes
-	 * @throws IOException
-	 *         thrown if an error occurs while reading the data
-	 */
-	private void testReceivedData(final InputStream inputStream, final int expectedSize) throws IOException {
-
-		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
-
-		int totalBytesRead = 0;
-		int nextExpectedNumber = 0;
-		while (true) {
-
-			final int bytesRead = inputStream.read(testBuffer);
-			if (bytesRead < 0) {
-				break;
-			}
-
-			totalBytesRead += bytesRead;
-
-			for (int i = 0; i < bytesRead; ++i) {
-				if (testBuffer[i] != nextExpectedNumber) {
-					throw new IOException("Read number " + testBuffer[i] + " but expected " + nextExpectedNumber);
-				}
-
-				++nextExpectedNumber;
-
-				if (nextExpectedNumber == MODULUS) {
-					nextExpectedNumber = 0;
-				}
-			}
-		}
-
-		if (totalBytesRead != expectedSize) {
-			throw new IOException("Expected to read " + expectedSize + " bytes but only received " + totalBytesRead);
-		}
-	}
-
-	/**
-	 * Generates test data of the given size according to some specific pattern and writes it to the provided output
-	 * stream.
-	 * 
-	 * @param outputStream
-	 *        the output stream to write the data to
-	 * @param size
-	 *        the size of the test data to be generated in bytes
-	 * @throws IOException
-	 *         thrown if an error occurs while writing the data
-	 */
-	private void generateTestData(final OutputStream outputStream, final int size) throws IOException {
-
-		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
-		for (int i = 0; i < testBuffer.length; ++i) {
-			testBuffer[i] = (byte) (i % MODULUS);
-		}
-
-		int bytesWritten = 0;
-		while (bytesWritten < size) {
-
-			final int diff = size - bytesWritten;
-			if (diff < testBuffer.length) {
-				outputStream.write(testBuffer, 0, diff);
-				bytesWritten += diff;
-			} else {
-				outputStream.write(testBuffer);
-				bytesWritten += testBuffer.length;
-			}
-		}
-	}
-
-	/**
-	 * Generates a random name.
-	 * 
-	 * @return a random name
-	 */
-	private String getRandomName() {
-
-		final StringBuilder stringBuilder = new StringBuilder();
-		for (int i = 0; i < NAME_LENGTH; ++i) {
-			final char c = ALPHABET[(int) (Math.random() * (double) ALPHABET.length)];
-			stringBuilder.append(c);
-		}
-
-		return stringBuilder.toString();
-	}
-
-	/**
-	 * Checks whether the AWS access key and the AWS secret keys have been successfully loaded from the configuration
-	 * and whether the S3 tests shall be performed.
-	 * 
-	 * @return <code>true</code> if the tests shall be performed, <code>false</code> if the tests shall be skipped
-	 *         because at least one AWS key is missing
-	 */
-	private boolean testActivated() {
-
-		final String accessKey = GlobalConfiguration.getString(S3FileSystem.S3_ACCESS_KEY_KEY, null);
-		final String secretKey = GlobalConfiguration.getString(S3FileSystem.S3_SECRET_KEY_KEY, null);
-
-		if (accessKey != null && secretKey != null) {
-			return true;
-		}
-
-		return false;
-	}
-}


[09/10] flink git commit: [FLINK-2107] Add hash-based strategies for left and right outer joins.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashJoinIterator.java
new file mode 100644
index 0000000..3b940c2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashJoinIterator.java
@@ -0,0 +1,166 @@
+/*
+ * 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.runtime.operators.hash;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.util.JoinTaskIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.List;
+
+
+/**
+ * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
+ * internally to match the records with equal key. The build side of the hash is the first input of the match.
+ * This implementation DOES NOT reuse objects.
+ */
+public class NonReusingBuildFirstHashJoinIterator<V1, V2, O> extends HashJoinIteratorBase implements JoinTaskIterator<V1, V2, O> {
+
+	protected final MutableHashTable<V1, V2> hashJoin;
+
+	protected final TypeSerializer<V2> probeSideSerializer;
+
+	private final MemoryManager memManager;
+
+	private final MutableObjectIterator<V1> firstInput;
+
+	private final MutableObjectIterator<V2> secondInput;
+
+	private final boolean joinWithEmptyBuildSide;
+
+	private volatile boolean running = true;
+
+	// --------------------------------------------------------------------------------------------
+
+	public NonReusingBuildFirstHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V2, V1> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		this.memManager = memManager;
+		this.firstInput = firstInput;
+		this.secondInput = secondInput;
+		this.probeSideSerializer = serializer2;
+
+		if(useBitmapFilters && joinWithEmptyBuildSide) {
+			throw new IllegalArgumentException("Bitmap filter may not be activated for joining with empty build side");
+		}
+		this.joinWithEmptyBuildSide = joinWithEmptyBuildSide;
+
+		this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2,
+				pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void open() throws IOException, MemoryAllocationException, InterruptedException {
+		this.hashJoin.open(this.firstInput, this.secondInput);
+	}
+	
+
+	@Override
+	public void close() {
+		// close the join
+		this.hashJoin.close();
+		
+		// free the memory
+		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
+		this.memManager.release(segments);
+	}
+
+	@Override
+	public final boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
+	throws Exception
+	{
+		if (this.hashJoin.nextRecord())
+		{
+			// we have a next record, get the iterators to the probe and build side values
+			final MutableHashTable.HashBucketIterator<V1, V2> buildSideIterator = this.hashJoin.getBuildSideIterator();
+			V1 nextBuildSideRecord;
+			
+			// get the first build side value
+			if ((nextBuildSideRecord = buildSideIterator.next()) != null) {
+				V1 tmpRec;
+				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
+				
+				// check if there is another build-side value
+				if ((tmpRec = buildSideIterator.next()) != null) {
+					// more than one build-side value --> copy the probe side
+					V2 probeCopy;
+					probeCopy = this.probeSideSerializer.copy(probeRecord);
+					
+					// call match on the first pair
+					matchFunction.join(nextBuildSideRecord, probeCopy, collector);
+					
+					// call match on the second pair
+					probeCopy = this.probeSideSerializer.copy(probeRecord);
+					matchFunction.join(tmpRec, probeCopy, collector);
+					
+					while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) {
+						// call match on the next pair
+						// make sure we restore the value of the probe side record
+						probeCopy = this.probeSideSerializer.copy(probeRecord);
+						matchFunction.join(nextBuildSideRecord, probeCopy, collector);
+					}
+				}
+				else {
+					// only single pair matches
+					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
+				}
+			}
+			else if(joinWithEmptyBuildSide) {
+				// build side is empty, join with null
+				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
+
+				matchFunction.join(null, probeRecord, collector);
+			}
+			return true;
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public void abort() {
+		this.running = false;
+		this.hashJoin.abort();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
deleted file mode 100644
index dbdb5b2..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
+++ /dev/null
@@ -1,152 +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.runtime.operators.hash;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.util.JoinTaskIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-import java.io.IOException;
-import java.util.List;
-
-
-/**
- * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
- * internally to match the records with equal key. The build side of the hash is the first input of the match.
- * This implementation DOES NOT reuse objects.
- */
-public class NonReusingBuildFirstHashMatchIterator<V1, V2, O> extends HashMatchIteratorBase implements JoinTaskIterator<V1, V2, O> {
-
-	protected final MutableHashTable<V1, V2> hashJoin;
-
-	protected final TypeSerializer<V2> probeSideSerializer;
-
-	private final MemoryManager memManager;
-
-	private final MutableObjectIterator<V1> firstInput;
-
-	private final MutableObjectIterator<V2> secondInput;
-
-	private volatile boolean running = true;
-
-	// --------------------------------------------------------------------------------------------
-
-	public NonReusingBuildFirstHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V2, V1> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		this.memManager = memManager;
-		this.firstInput = firstInput;
-		this.secondInput = secondInput;
-		this.probeSideSerializer = serializer2;
-
-		this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2,
-				pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void open() throws IOException, MemoryAllocationException, InterruptedException {
-		this.hashJoin.open(this.firstInput, this.secondInput);
-	}
-	
-
-	@Override
-	public void close() {
-		// close the join
-		this.hashJoin.close();
-		
-		// free the memory
-		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
-		this.memManager.release(segments);
-	}
-
-	@Override
-	public final boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
-	throws Exception
-	{
-		if (this.hashJoin.nextRecord())
-		{
-			// we have a next record, get the iterators to the probe and build side values
-			final MutableHashTable.HashBucketIterator<V1, V2> buildSideIterator = this.hashJoin.getBuildSideIterator();
-			V1 nextBuildSideRecord;
-			
-			// get the first build side value
-			if ((nextBuildSideRecord = buildSideIterator.next()) != null) {
-				V1 tmpRec;
-				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
-				
-				// check if there is another build-side value
-				if ((tmpRec = buildSideIterator.next()) != null) {
-					// more than one build-side value --> copy the probe side
-					V2 probeCopy;
-					probeCopy = this.probeSideSerializer.copy(probeRecord);
-					
-					// call match on the first pair
-					matchFunction.join(nextBuildSideRecord, probeCopy, collector);
-					
-					// call match on the second pair
-					probeCopy = this.probeSideSerializer.copy(probeRecord);
-					matchFunction.join(tmpRec, probeCopy, collector);
-					
-					while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) {
-						// call match on the next pair
-						// make sure we restore the value of the probe side record
-						probeCopy = this.probeSideSerializer.copy(probeRecord);
-						matchFunction.join(nextBuildSideRecord, probeCopy, collector);
-					}
-				}
-				else {
-					// only single pair matches
-					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
-				}
-			}
-			return true;
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public void abort() {
-		this.running = false;
-		this.hashJoin.abort();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashJoinIterator.java
new file mode 100644
index 0000000..77521af
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashJoinIterator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.operators.hash;
+
+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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.List;
+
+public class NonReusingBuildFirstReOpenableHashJoinIterator<V1, V2, O> extends NonReusingBuildFirstHashJoinIterator<V1, V2, O> {
+
+
+	private final ReOpenableMutableHashTable<V1, V2> reopenHashTable;
+
+	public NonReusingBuildFirstReOpenableHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V2, V1> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		super(firstInput, secondInput, serializer1, comparator1, serializer2,
+				comparator2, pairComparator, memManager, ioManager, ownerTask,
+				memoryFraction, joinWithEmptyBuildSide, useBitmapFilters);
+		
+		reopenHashTable = (ReOpenableMutableHashTable<V1, V2>) hashJoin;
+	}
+
+	@Override
+	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
+			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
+			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
+			TypePairComparator<PT, BT> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
+		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
+		
+		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
+				buildSideComparator, probeSideComparator, pairComparator,
+				memorySegments, ioManager, useBitmapFilters);
+	}
+
+	/**
+	 * Set new input for probe side
+	 * @throws java.io.IOException
+	 */
+	public void reopenProbe(MutableObjectIterator<V2> probeInput) throws IOException {
+		reopenHashTable.reopenProbe(probeInput);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
deleted file mode 100644
index b51c3b1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
+++ /dev/null
@@ -1,86 +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.runtime.operators.hash;
-
-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.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.util.MutableObjectIterator;
-
-import java.io.IOException;
-import java.util.List;
-
-public class NonReusingBuildFirstReOpenableHashMatchIterator<V1, V2, O> extends NonReusingBuildFirstHashMatchIterator<V1, V2, O> {
-
-
-	private final ReOpenableMutableHashTable<V1, V2> reopenHashTable;
-
-	public NonReusingBuildFirstReOpenableHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V2, V1> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		super(firstInput, secondInput, serializer1, comparator1, serializer2,
-				comparator2, pairComparator, memManager, ioManager, ownerTask,
-				memoryFraction, useBitmapFilters);
-		
-		reopenHashTable = (ReOpenableMutableHashTable<V1, V2>) hashJoin;
-	}
-
-	@Override
-	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
-			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
-			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
-			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		final int numPages = memManager.computeNumberOfPages(memoryFraction);
-		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
-		
-		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
-				buildSideComparator, probeSideComparator, pairComparator,
-				memorySegments, ioManager, useBitmapFilters);
-	}
-
-	/**
-	 * Set new input for probe side
-	 * @throws java.io.IOException
-	 */
-	public void reopenProbe(MutableObjectIterator<V2> probeInput) throws IOException {
-		reopenHashTable.reopenProbe(probeInput);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashJoinIterator.java
new file mode 100644
index 0000000..9ea0b74
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashJoinIterator.java
@@ -0,0 +1,164 @@
+/*
+ * 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.runtime.operators.hash;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.util.JoinTaskIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.List;
+
+
+/**
+ * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
+ * internally to match the records with equal key. The build side of the hash is the second input of the match.  
+ */
+public class NonReusingBuildSecondHashJoinIterator<V1, V2, O> extends HashJoinIteratorBase implements JoinTaskIterator<V1, V2, O> {
+
+	protected final MutableHashTable<V2, V1> hashJoin;
+
+	protected final TypeSerializer<V1> probeSideSerializer;
+
+	private final MemoryManager memManager;
+
+	private final MutableObjectIterator<V1> firstInput;
+
+	private final MutableObjectIterator<V2> secondInput;
+
+	private final boolean joinWithEmptyBuildSide;
+
+	private volatile boolean running = true;
+
+	// --------------------------------------------------------------------------------------------
+
+	public NonReusingBuildSecondHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V1, V2> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		this.memManager = memManager;
+		this.firstInput = firstInput;
+		this.secondInput = secondInput;
+		this.probeSideSerializer = serializer1;
+
+		if(useBitmapFilters && joinWithEmptyBuildSide) {
+			throw new IllegalArgumentException("Bitmap filter may not be activated for joining with empty build side");
+		}
+		this.joinWithEmptyBuildSide = joinWithEmptyBuildSide;
+		
+		this.hashJoin = getHashJoin(serializer2, comparator2, serializer1,
+				comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void open() throws IOException, MemoryAllocationException, InterruptedException {
+		this.hashJoin.open(this.secondInput, this.firstInput);
+	}
+
+	@Override
+	public void close() {
+		// close the join
+		this.hashJoin.close();
+		
+		// free the memory
+		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
+		this.memManager.release(segments);
+	}
+
+	@Override
+	public boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
+	throws Exception
+	{
+		if (this.hashJoin.nextRecord())
+		{
+			// we have a next record, get the iterators to the probe and build side values
+			final MutableHashTable.HashBucketIterator<V2, V1> buildSideIterator = this.hashJoin.getBuildSideIterator();
+			V2 nextBuildSideRecord;
+			
+			// get the first build side value
+			if ((nextBuildSideRecord = buildSideIterator.next()) != null) {
+				V2 tmpRec;
+				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
+				
+				// check if there is another build-side value
+				if ((tmpRec = buildSideIterator.next()) != null) {
+					// more than one build-side value --> copy the probe side
+					V1 probeCopy;
+					probeCopy = this.probeSideSerializer.copy(probeRecord);
+
+					// call match on the first pair
+					matchFunction.join(probeCopy, nextBuildSideRecord, collector);
+					
+					// call match on the second pair
+					probeCopy = this.probeSideSerializer.copy(probeRecord);
+					matchFunction.join(probeCopy, tmpRec, collector);
+					
+					while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) {
+						// call match on the next pair
+						// make sure we restore the value of the probe side record
+						probeCopy = this.probeSideSerializer.copy(probeRecord);
+						matchFunction.join(probeCopy, nextBuildSideRecord, collector);
+					}
+				}
+				else {
+					// only single pair matches
+					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
+				}
+			}
+			else if(joinWithEmptyBuildSide) {
+				// build side is empty, join with null
+				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
+
+				matchFunction.join(probeRecord, null, collector);
+			}
+			return true;
+		}
+		else {
+			return false;
+		}
+	}
+	
+	@Override
+	public void abort() {
+		this.running = false;
+		this.hashJoin.abort();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
deleted file mode 100644
index 26dba7b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
+++ /dev/null
@@ -1,150 +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.runtime.operators.hash;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.util.JoinTaskIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-import java.io.IOException;
-import java.util.List;
-
-
-/**
- * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
- * internally to match the records with equal key. The build side of the hash is the second input of the match.  
- */
-public class NonReusingBuildSecondHashMatchIterator<V1, V2, O> extends HashMatchIteratorBase implements JoinTaskIterator<V1, V2, O> {
-
-	protected final MutableHashTable<V2, V1> hashJoin;
-
-	protected final TypeSerializer<V1> probeSideSerializer;
-
-	private final MemoryManager memManager;
-
-	private final MutableObjectIterator<V1> firstInput;
-
-	private final MutableObjectIterator<V2> secondInput;
-
-	private volatile boolean running = true;
-
-	// --------------------------------------------------------------------------------------------
-
-	public NonReusingBuildSecondHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V1, V2> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		this.memManager = memManager;
-		this.firstInput = firstInput;
-		this.secondInput = secondInput;
-		this.probeSideSerializer = serializer1;
-		
-		this.hashJoin = getHashJoin(serializer2, comparator2, serializer1,
-				comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void open() throws IOException, MemoryAllocationException, InterruptedException {
-		this.hashJoin.open(this.secondInput, this.firstInput);
-	}
-
-	@Override
-	public void close() {
-		// close the join
-		this.hashJoin.close();
-		
-		// free the memory
-		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
-		this.memManager.release(segments);
-	}
-
-	@Override
-	public boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
-	throws Exception
-	{
-		if (this.hashJoin.nextRecord())
-		{
-			// we have a next record, get the iterators to the probe and build side values
-			final MutableHashTable.HashBucketIterator<V2, V1> buildSideIterator = this.hashJoin.getBuildSideIterator();
-			V2 nextBuildSideRecord;
-			
-			// get the first build side value
-			if ((nextBuildSideRecord = buildSideIterator.next()) != null) {
-				V2 tmpRec;
-				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
-				
-				// check if there is another build-side value
-				if ((tmpRec = buildSideIterator.next()) != null) {
-					// more than one build-side value --> copy the probe side
-					V1 probeCopy;
-					probeCopy = this.probeSideSerializer.copy(probeRecord);
-
-					// call match on the first pair
-					matchFunction.join(probeCopy, nextBuildSideRecord, collector);
-					
-					// call match on the second pair
-					probeCopy = this.probeSideSerializer.copy(probeRecord);
-					matchFunction.join(probeCopy, tmpRec, collector);
-					
-					while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) {
-						// call match on the next pair
-						// make sure we restore the value of the probe side record
-						probeCopy = this.probeSideSerializer.copy(probeRecord);
-						matchFunction.join(probeCopy, nextBuildSideRecord, collector);
-					}
-				}
-				else {
-					// only single pair matches
-					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
-				}
-			}
-			return true;
-		}
-		else {
-			return false;
-		}
-	}
-	
-	@Override
-	public void abort() {
-		this.running = false;
-		this.hashJoin.abort();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashJoinIterator.java
new file mode 100644
index 0000000..c9c9165
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashJoinIterator.java
@@ -0,0 +1,88 @@
+/*
+ * 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.runtime.operators.hash;
+
+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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.List;
+
+public class NonReusingBuildSecondReOpenableHashJoinIterator<V1, V2, O> extends NonReusingBuildSecondHashJoinIterator<V1, V2, O> {
+
+
+	private final ReOpenableMutableHashTable<V2, V1> reopenHashTable;
+
+	public NonReusingBuildSecondReOpenableHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V1, V2> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		super(firstInput, secondInput, serializer1, comparator1, serializer2,
+				comparator2, pairComparator, memManager, ioManager, ownerTask,
+				memoryFraction, joinWithEmptyBuildSide, useBitmapFilters);
+		
+		reopenHashTable = (ReOpenableMutableHashTable<V2, V1>) hashJoin;
+	}
+
+	@Override
+	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
+			TypeSerializer<BT> buildSideSerializer,
+			TypeComparator<BT> buildSideComparator,
+			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
+			TypePairComparator<PT, BT> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
+		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
+		
+		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
+				buildSideComparator, probeSideComparator, pairComparator,
+				memorySegments, ioManager, useBitmapFilters);
+	}
+
+	/**
+	 * Set new input for probe side
+	 * @throws java.io.IOException
+	 */
+	public void reopenProbe(MutableObjectIterator<V1> probeInput) throws IOException {
+		reopenHashTable.reopenProbe(probeInput);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
deleted file mode 100644
index 92b0fff..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
+++ /dev/null
@@ -1,86 +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.runtime.operators.hash;
-
-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.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.util.MutableObjectIterator;
-
-import java.io.IOException;
-import java.util.List;
-
-public class NonReusingBuildSecondReOpenableHashMatchIterator<V1, V2, O> extends NonReusingBuildSecondHashMatchIterator<V1, V2, O> {
-
-
-	private final ReOpenableMutableHashTable<V2, V1> reopenHashTable;
-
-	public NonReusingBuildSecondReOpenableHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V1, V2> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		super(firstInput, secondInput, serializer1, comparator1, serializer2,
-				comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-		
-		reopenHashTable = (ReOpenableMutableHashTable<V2, V1>) hashJoin;
-	}
-
-	@Override
-	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
-			TypeSerializer<BT> buildSideSerializer,
-			TypeComparator<BT> buildSideComparator,
-			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
-			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		final int numPages = memManager.computeNumberOfPages(memoryFraction);
-		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
-		
-		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
-				buildSideComparator, probeSideComparator, pairComparator,
-				memorySegments, ioManager, useBitmapFilters);
-	}
-
-	/**
-	 * Set new input for probe side
-	 * @throws java.io.IOException
-	 */
-	public void reopenProbe(MutableObjectIterator<V1> probeInput) throws IOException {
-		reopenHashTable.reopenProbe(probeInput);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashJoinIterator.java
new file mode 100644
index 0000000..c1e601d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashJoinIterator.java
@@ -0,0 +1,168 @@
+/*
+ * 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.runtime.operators.hash;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.util.JoinTaskIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+
+/**
+ * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
+ * internally to match the records with equal key. The build side of the hash is the first input of the match.
+ */
+public class ReusingBuildFirstHashJoinIterator<V1, V2, O> extends HashJoinIteratorBase implements JoinTaskIterator<V1, V2, O> {
+	
+	protected final MutableHashTable<V1, V2> hashJoin;
+	
+	private final V1 nextBuildSideObject;
+
+	private final V1 tempBuildSideRecord;
+
+	protected final TypeSerializer<V2> probeSideSerializer;
+	
+	private final MemoryManager memManager;
+	
+	private final MutableObjectIterator<V1> firstInput;
+	
+	private final MutableObjectIterator<V2> secondInput;
+
+	private final boolean joinWithEmptyBuildSide;
+	
+	private volatile boolean running = true;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public ReusingBuildFirstHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V2, V1> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		this.memManager = memManager;
+		this.firstInput = firstInput;
+		this.secondInput = secondInput;
+		this.probeSideSerializer = serializer2;
+
+		if(useBitmapFilters && joinWithEmptyBuildSide) {
+			throw new IllegalArgumentException("Bitmap filter may not be activated for joining with empty build side");
+		}
+		this.joinWithEmptyBuildSide = joinWithEmptyBuildSide;
+		
+		this.nextBuildSideObject = serializer1.createInstance();
+		this.tempBuildSideRecord = serializer1.createInstance();
+
+		this.hashJoin = getHashJoin(serializer1, comparator1, serializer2,
+				comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void open() throws IOException, MemoryAllocationException, InterruptedException {
+		this.hashJoin.open(this.firstInput, this.secondInput);
+	}
+	
+
+	@Override
+	public void close() {
+		// close the join
+		this.hashJoin.close();
+		
+		// free the memory
+		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
+		this.memManager.release(segments);
+	}
+
+	@Override
+	public final boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
+	throws Exception
+	{
+		if (this.hashJoin.nextRecord())
+		{
+			// we have a next record, get the iterators to the probe and build side values
+			final MutableHashTable.HashBucketIterator<V1, V2> buildSideIterator = this.hashJoin.getBuildSideIterator();
+			V1 nextBuildSideRecord = this.nextBuildSideObject;
+			
+			// get the first build side value
+			if ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null) {
+				V1 tmpRec = this.tempBuildSideRecord;
+				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
+				
+				// check if there is another build-side value
+				if ((tmpRec = buildSideIterator.next(tmpRec)) != null) {
+
+					// call match on the first pair
+					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
+					
+					// call match on the second pair
+					matchFunction.join(tmpRec, probeRecord, collector);
+					
+					while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) {
+						// call match on the next pair
+						// make sure we restore the value of the probe side record
+						matchFunction.join(nextBuildSideRecord, probeRecord, collector);
+					}
+				}
+				else {
+					// only single pair matches
+					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
+				}
+			}
+			else if(joinWithEmptyBuildSide) {
+				// build side is empty, join with null
+				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
+
+				matchFunction.join(null, probeRecord, collector);
+			}
+			return true;
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public void abort() {
+		this.running = false;
+		this.hashJoin.abort();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
deleted file mode 100644
index 65dfd89..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.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.runtime.operators.hash;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.util.JoinTaskIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-
-/**
- * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
- * internally to match the records with equal key. The build side of the hash is the first input of the match.
- */
-public class ReusingBuildFirstHashMatchIterator<V1, V2, O> extends HashMatchIteratorBase implements JoinTaskIterator<V1, V2, O> {
-	
-	protected final MutableHashTable<V1, V2> hashJoin;
-	
-	private final V1 nextBuildSideObject;
-
-	private final V1 tempBuildSideRecord;
-
-	protected final TypeSerializer<V2> probeSideSerializer;
-	
-	private final MemoryManager memManager;
-	
-	private final MutableObjectIterator<V1> firstInput;
-	
-	private final MutableObjectIterator<V2> secondInput;
-	
-	private volatile boolean running = true;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public ReusingBuildFirstHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V2, V1> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		this.memManager = memManager;
-		this.firstInput = firstInput;
-		this.secondInput = secondInput;
-		this.probeSideSerializer = serializer2;
-		
-		this.nextBuildSideObject = serializer1.createInstance();
-		this.tempBuildSideRecord = serializer1.createInstance();
-
-		this.hashJoin = getHashJoin(serializer1, comparator1, serializer2,
-				comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void open() throws IOException, MemoryAllocationException, InterruptedException {
-		this.hashJoin.open(this.firstInput, this.secondInput);
-	}
-	
-
-	@Override
-	public void close() {
-		// close the join
-		this.hashJoin.close();
-		
-		// free the memory
-		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
-		this.memManager.release(segments);
-	}
-
-	@Override
-	public final boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
-	throws Exception
-	{
-		if (this.hashJoin.nextRecord())
-		{
-			// we have a next record, get the iterators to the probe and build side values
-			final MutableHashTable.HashBucketIterator<V1, V2> buildSideIterator = this.hashJoin.getBuildSideIterator();
-			V1 nextBuildSideRecord = this.nextBuildSideObject;
-			
-			// get the first build side value
-			if ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null) {
-				V1 tmpRec = this.tempBuildSideRecord;
-				final V2 probeRecord = this.hashJoin.getCurrentProbeRecord();
-				
-				// check if there is another build-side value
-				if ((tmpRec = buildSideIterator.next(tmpRec)) != null) {
-
-					// call match on the first pair
-					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
-					
-					// call match on the second pair
-					matchFunction.join(tmpRec, probeRecord, collector);
-					
-					while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) {
-						// call match on the next pair
-						// make sure we restore the value of the probe side record
-						matchFunction.join(nextBuildSideRecord, probeRecord, collector);
-					}
-				}
-				else {
-					// only single pair matches
-					matchFunction.join(nextBuildSideRecord, probeRecord, collector);
-				}
-			}
-			return true;
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public void abort() {
-		this.running = false;
-		this.hashJoin.abort();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashJoinIterator.java
new file mode 100644
index 0000000..1cc3f91
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashJoinIterator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.operators.hash;
+
+import java.io.IOException;
+import java.util.List;
+
+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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class ReusingBuildFirstReOpenableHashJoinIterator<V1, V2, O> extends ReusingBuildFirstHashJoinIterator<V1, V2, O> {
+
+	
+	private final ReOpenableMutableHashTable<V1, V2> reopenHashTable;
+	
+	public ReusingBuildFirstReOpenableHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V2, V1> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters)
+		throws MemoryAllocationException
+	{
+		super(firstInput, secondInput, serializer1, comparator1, serializer2,
+				comparator2, pairComparator, memManager, ioManager, ownerTask,
+				memoryFraction, joinWithEmptyBuildSide, useBitmapFilters);
+		reopenHashTable = (ReOpenableMutableHashTable<V1, V2>) hashJoin;
+	}
+
+	@Override
+	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
+			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
+			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
+			TypePairComparator<PT, BT> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
+		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
+		
+		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
+				buildSideComparator, probeSideComparator, pairComparator,
+				memorySegments, ioManager, useBitmapFilters);
+	}
+	
+	/**
+	 * Set new input for probe side
+	 * @throws IOException 
+	 */
+	public void reopenProbe(MutableObjectIterator<V2> probeInput) throws IOException {
+		reopenHashTable.reopenProbe(probeInput);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
deleted file mode 100644
index 5635865..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
+++ /dev/null
@@ -1,86 +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.runtime.operators.hash;
-
-import java.io.IOException;
-import java.util.List;
-
-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.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.util.MutableObjectIterator;
-
-public class ReusingBuildFirstReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildFirstHashMatchIterator<V1, V2, O> {
-
-	
-	private final ReOpenableMutableHashTable<V1, V2> reopenHashTable;
-	
-	public ReusingBuildFirstReOpenableHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V2, V1> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters)
-		throws MemoryAllocationException
-	{
-		super(firstInput, secondInput, serializer1, comparator1, serializer2,
-				comparator2, pairComparator, memManager, ioManager, ownerTask,
-				memoryFraction, useBitmapFilters);
-		reopenHashTable = (ReOpenableMutableHashTable<V1, V2>) hashJoin;
-	}
-
-	@Override
-	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
-			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
-			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
-			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		final int numPages = memManager.computeNumberOfPages(memoryFraction);
-		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
-		
-		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
-				buildSideComparator, probeSideComparator, pairComparator,
-				memorySegments, ioManager, useBitmapFilters);
-	}
-	
-	/**
-	 * Set new input for probe side
-	 * @throws IOException 
-	 */
-	public void reopenProbe(MutableObjectIterator<V2> probeInput) throws IOException {
-		reopenHashTable.reopenProbe(probeInput);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashJoinIterator.java
new file mode 100644
index 0000000..4402665
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashJoinIterator.java
@@ -0,0 +1,166 @@
+/*
+ * 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.runtime.operators.hash;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.util.JoinTaskIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+
+/**
+ * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
+ * internally to match the records with equal key. The build side of the hash is the second input of the match.  
+ */
+public class ReusingBuildSecondHashJoinIterator<V1, V2, O> extends HashJoinIteratorBase implements JoinTaskIterator<V1, V2, O> {
+	
+	protected final MutableHashTable<V2, V1> hashJoin;
+	
+	private final V2 nextBuildSideObject;
+	
+	private final V2 tempBuildSideRecord;
+	
+	protected final TypeSerializer<V1> probeSideSerializer;
+	
+	private final MemoryManager memManager;
+	
+	private final MutableObjectIterator<V1> firstInput;
+	
+	private final MutableObjectIterator<V2> secondInput;
+
+	private final boolean joinWithEmptyBuildSide;
+	
+	private volatile boolean running = true;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public ReusingBuildSecondHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V1, V2> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		this.memManager = memManager;
+		this.firstInput = firstInput;
+		this.secondInput = secondInput;
+		this.probeSideSerializer = serializer1;
+
+		if(useBitmapFilters && joinWithEmptyBuildSide) {
+			throw new IllegalArgumentException("Bitmap filter may not be activated for joining with empty build side");
+		}
+		this.joinWithEmptyBuildSide = joinWithEmptyBuildSide;
+		
+		this.nextBuildSideObject = serializer2.createInstance();
+		this.tempBuildSideRecord = serializer2.createInstance();
+
+		this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, comparator1, pairComparator,
+			memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void open() throws IOException, MemoryAllocationException, InterruptedException {
+		this.hashJoin.open(this.secondInput, this.firstInput);
+	}
+
+	@Override
+	public void close() {
+		// close the join
+		this.hashJoin.close();
+		
+		// free the memory
+		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
+		this.memManager.release(segments);
+	}
+
+	@Override
+	public boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
+	throws Exception
+	{
+		if (this.hashJoin.nextRecord())
+		{
+			// we have a next record, get the iterators to the probe and build side values
+			final MutableHashTable.HashBucketIterator<V2, V1> buildSideIterator = this.hashJoin.getBuildSideIterator();
+			V2 nextBuildSideRecord = this.nextBuildSideObject;
+			
+			// get the first build side value
+			if ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null) {
+				V2 tmpRec = this.tempBuildSideRecord;
+				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
+				
+				// check if there is another build-side value
+				if ((tmpRec = buildSideIterator.next(tmpRec)) != null) {
+					// call match on the first pair
+					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
+					
+					// call match on the second pair
+					matchFunction.join(probeRecord, tmpRec, collector);
+					
+					while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) {
+						// call match on the next pair
+						// make sure we restore the value of the probe side record
+						matchFunction.join(probeRecord, nextBuildSideRecord, collector);
+					}
+				}
+				else {
+					// only single pair matches
+					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
+				}
+			}
+			else if(joinWithEmptyBuildSide) {
+				// build side is empty, join with null
+				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
+
+				matchFunction.join(probeRecord, null, collector);
+			}
+			return true;
+		}
+		else {
+			return false;
+		}
+	}
+	
+	@Override
+	public void abort() {
+		this.running = false;
+		this.hashJoin.abort();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
deleted file mode 100644
index 156f259..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
+++ /dev/null
@@ -1,152 +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.runtime.operators.hash;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.util.JoinTaskIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-
-/**
- * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join
- * internally to match the records with equal key. The build side of the hash is the second input of the match.  
- */
-public class ReusingBuildSecondHashMatchIterator<V1, V2, O> extends HashMatchIteratorBase implements JoinTaskIterator<V1, V2, O> {
-	
-	protected final MutableHashTable<V2, V1> hashJoin;
-	
-	private final V2 nextBuildSideObject;
-	
-	private final V2 tempBuildSideRecord;
-	
-	protected final TypeSerializer<V1> probeSideSerializer;
-	
-	private final MemoryManager memManager;
-	
-	private final MutableObjectIterator<V1> firstInput;
-	
-	private final MutableObjectIterator<V2> secondInput;
-	
-	private volatile boolean running = true;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public ReusingBuildSecondHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V1, V2> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		this.memManager = memManager;
-		this.firstInput = firstInput;
-		this.secondInput = secondInput;
-		this.probeSideSerializer = serializer1;
-		
-		this.nextBuildSideObject = serializer2.createInstance();
-		this.tempBuildSideRecord = serializer2.createInstance();
-
-		this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, comparator1, pairComparator,
-			memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void open() throws IOException, MemoryAllocationException, InterruptedException {
-		this.hashJoin.open(this.secondInput, this.firstInput);
-	}
-
-	@Override
-	public void close() {
-		// close the join
-		this.hashJoin.close();
-		
-		// free the memory
-		final List<MemorySegment> segments = this.hashJoin.getFreedMemory();
-		this.memManager.release(segments);
-	}
-
-	@Override
-	public boolean callWithNextKey(FlatJoinFunction<V1, V2, O> matchFunction, Collector<O> collector)
-	throws Exception
-	{
-		if (this.hashJoin.nextRecord())
-		{
-			// we have a next record, get the iterators to the probe and build side values
-			final MutableHashTable.HashBucketIterator<V2, V1> buildSideIterator = this.hashJoin.getBuildSideIterator();
-			V2 nextBuildSideRecord = this.nextBuildSideObject;
-			
-			// get the first build side value
-			if ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null) {
-				V2 tmpRec = this.tempBuildSideRecord;
-				final V1 probeRecord = this.hashJoin.getCurrentProbeRecord();
-				
-				// check if there is another build-side value
-				if ((tmpRec = buildSideIterator.next(tmpRec)) != null) {
-					// call match on the first pair
-					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
-					
-					// call match on the second pair
-					matchFunction.join(probeRecord, tmpRec, collector);
-					
-					while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) {
-						// call match on the next pair
-						// make sure we restore the value of the probe side record
-						matchFunction.join(probeRecord, nextBuildSideRecord, collector);
-					}
-				}
-				else {
-					// only single pair matches
-					matchFunction.join(probeRecord, nextBuildSideRecord, collector);
-				}
-			}
-			return true;
-		}
-		else {
-			return false;
-		}
-	}
-	
-	@Override
-	public void abort() {
-		this.running = false;
-		this.hashJoin.abort();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashJoinIterator.java
new file mode 100644
index 0000000..190398f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashJoinIterator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.operators.hash;
+
+import java.io.IOException;
+import java.util.List;
+
+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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class ReusingBuildSecondReOpenableHashJoinIterator<V1, V2, O> extends ReusingBuildSecondHashJoinIterator<V1, V2, O> {
+
+	
+	private final ReOpenableMutableHashTable<V2, V1> reopenHashTable;
+	
+	public ReusingBuildSecondReOpenableHashJoinIterator(
+			MutableObjectIterator<V1> firstInput,
+			MutableObjectIterator<V2> secondInput,
+			TypeSerializer<V1> serializer1,
+			TypeComparator<V1> comparator1,
+			TypeSerializer<V2> serializer2,
+			TypeComparator<V2> comparator2,
+			TypePairComparator<V1, V2> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean joinWithEmptyBuildSide,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		super(firstInput, secondInput, serializer1, comparator1, serializer2,
+				comparator2, pairComparator, memManager, ioManager, ownerTask,
+				memoryFraction, joinWithEmptyBuildSide, useBitmapFilters);
+		
+		reopenHashTable = (ReOpenableMutableHashTable<V2, V1>) hashJoin;
+	}
+
+	@Override
+	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
+			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
+			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
+			TypePairComparator<PT, BT> pairComparator,
+			MemoryManager memManager, IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean useBitmapFilters) throws MemoryAllocationException {
+		
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
+		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
+		
+		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
+				buildSideComparator, probeSideComparator, pairComparator,
+				memorySegments, ioManager, useBitmapFilters);
+	}
+	
+	/**
+	 * Set new input for probe side
+	 * @throws IOException 
+	 */
+	public void reopenProbe(MutableObjectIterator<V1> probeInput) throws IOException {
+		reopenHashTable.reopenProbe(probeInput);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
deleted file mode 100644
index a0791fe..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
+++ /dev/null
@@ -1,85 +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.runtime.operators.hash;
-
-import java.io.IOException;
-import java.util.List;
-
-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.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.util.MutableObjectIterator;
-
-public class ReusingBuildSecondReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildSecondHashMatchIterator<V1, V2, O> {
-
-	
-	private final ReOpenableMutableHashTable<V2, V1> reopenHashTable;
-	
-	public ReusingBuildSecondReOpenableHashMatchIterator(
-			MutableObjectIterator<V1> firstInput,
-			MutableObjectIterator<V2> secondInput,
-			TypeSerializer<V1> serializer1,
-			TypeComparator<V1> comparator1,
-			TypeSerializer<V2> serializer2,
-			TypeComparator<V2> comparator2,
-			TypePairComparator<V1, V2> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		super(firstInput, secondInput, serializer1, comparator1, serializer2,
-				comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction, useBitmapFilters);
-		
-		reopenHashTable = (ReOpenableMutableHashTable<V2, V1>) hashJoin;
-	}
-
-	@Override
-	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
-			TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
-			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
-			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBitmapFilters) throws MemoryAllocationException {
-		
-		final int numPages = memManager.computeNumberOfPages(memoryFraction);
-		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
-		
-		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
-				buildSideComparator, probeSideComparator, pairComparator,
-				memorySegments, ioManager, useBitmapFilters);
-	}
-	
-	/**
-	 * Set new input for probe side
-	 * @throws IOException 
-	 */
-	public void reopenProbe(MutableObjectIterator<V1> probeInput) throws IOException {
-		reopenHashTable.reopenProbe(probeInput);
-	}
-}


[10/10] flink git commit: [FLINK-2107] Add hash-based strategies for left and right outer joins.

Posted by fh...@apache.org.
[FLINK-2107] Add hash-based strategies for left and right outer joins.

This closes #1262


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

Branch: refs/heads/master
Commit: 5671c77c3afbb84ed89427bbab9b1355d0b1f8cc
Parents: c900577
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu Oct 15 10:58:58 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 19 16:17:42 2015 +0200

----------------------------------------------------------------------
 docs/apis/dataset_transformations.md            |  58 +-
 .../java/org/apache/flink/api/java/DataSet.java |  29 +-
 .../operator/FullOuterJoinOperatorTest.java     | 250 +++++
 .../operator/LeftOuterJoinOperatorTest.java     | 251 +++++
 .../operator/RightOuterJoinOperatorTest.java    | 250 +++++
 .../flink/optimizer/costs/CostEstimator.java    |   2 +
 .../flink/optimizer/dag/OuterJoinNode.java      |  97 +-
 .../HashLeftOuterJoinBuildSecondDescriptor.java |  68 ++
 .../HashRightOuterJoinBuildFirstDescriptor.java |  68 ++
 .../AbstractCachedBuildSideJoinDriver.java      |  28 +-
 .../operators/AbstractOuterJoinDriver.java      |  11 +-
 .../flink/runtime/operators/DriverStrategy.java |   8 +-
 .../runtime/operators/FullOuterJoinDriver.java  |   6 +-
 .../flink/runtime/operators/JoinDriver.java     |  20 +-
 .../runtime/operators/LeftOuterJoinDriver.java  |  28 +-
 .../runtime/operators/RightOuterJoinDriver.java |  28 +-
 .../operators/hash/HashJoinIteratorBase.java    |  56 ++
 .../operators/hash/HashMatchIteratorBase.java   |  56 --
 .../NonReusingBuildFirstHashJoinIterator.java   | 166 ++++
 .../NonReusingBuildFirstHashMatchIterator.java  | 152 ---
 ...ingBuildFirstReOpenableHashJoinIterator.java |  87 ++
 ...ngBuildFirstReOpenableHashMatchIterator.java |  86 --
 .../NonReusingBuildSecondHashJoinIterator.java  | 164 ++++
 .../NonReusingBuildSecondHashMatchIterator.java | 150 ---
 ...ngBuildSecondReOpenableHashJoinIterator.java |  88 ++
 ...gBuildSecondReOpenableHashMatchIterator.java |  86 --
 .../hash/ReusingBuildFirstHashJoinIterator.java | 168 ++++
 .../ReusingBuildFirstHashMatchIterator.java     | 154 ---
 ...ingBuildFirstReOpenableHashJoinIterator.java |  87 ++
 ...ngBuildFirstReOpenableHashMatchIterator.java |  86 --
 .../ReusingBuildSecondHashJoinIterator.java     | 166 ++++
 .../ReusingBuildSecondHashMatchIterator.java    | 152 ---
 ...ngBuildSecondReOpenableHashJoinIterator.java |  87 ++
 ...gBuildSecondReOpenableHashMatchIterator.java |  85 --
 .../AbstractOuterJoinTaskExternalITCase.java    |  23 +-
 .../operators/AbstractOuterJoinTaskTest.java    |  35 +-
 .../FullOuterJoinTaskExternalITCase.java        |   7 +-
 .../operators/FullOuterJoinTaskTest.java        |   7 +-
 .../LeftOuterJoinTaskExternalITCase.java        |  42 +-
 .../operators/LeftOuterJoinTaskTest.java        | 230 ++++-
 .../RightOuterJoinTaskExternalITCase.java       |  44 +-
 .../operators/RightOuterJoinTaskTest.java       | 223 ++++-
 .../hash/NonReusingHashJoinIteratorITCase.java  | 947 +++++++++++++++++++
 .../hash/NonReusingHashMatchIteratorITCase.java | 766 ---------------
 .../NonReusingReOpenableHashTableITCase.java    |  12 +-
 .../hash/ReusingHashJoinIteratorITCase.java     | 709 ++++++++++++++
 .../hash/ReusingHashMatchIteratorITCase.java    | 768 ---------------
 .../hash/ReusingReOpenableHashTableITCase.java  |  15 +-
 .../operators/util/HashVsSortMiniBenchmark.java |  16 +-
 .../org/apache/flink/api/scala/DataSet.scala    |  28 +-
 .../test/javaApiOperators/OuterJoinITCase.java  |  61 +-
 51 files changed, 4524 insertions(+), 2687 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/docs/apis/dataset_transformations.md
----------------------------------------------------------------------
diff --git a/docs/apis/dataset_transformations.md b/docs/apis/dataset_transformations.md
index cc7e742..feed121 100644
--- a/docs/apis/dataset_transformations.md
+++ b/docs/apis/dataset_transformations.md
@@ -1408,25 +1408,25 @@ Not supported.
 
 The following hints are available:
 
-* OPTIMIZER_CHOOSES: Equivalent to not giving a hint at all, leaves the choice to the system.
+* `OPTIMIZER_CHOOSES`: Equivalent to not giving a hint at all, leaves the choice to the system.
 
-* BROADCAST_HASH_FIRST: Broadcasts the first input and builds a hash table from it, which is
+* `BROADCAST_HASH_FIRST`: Broadcasts the first input and builds a hash table from it, which is
   probed by the second input. A good strategy if the first input is very small.
 
-* BROADCAST_HASH_SECOND: Broadcasts the second input and builds a hash table from it, which is
+* `BROADCAST_HASH_SECOND`: Broadcasts the second input and builds a hash table from it, which is
   probed by the first input. A good strategy if the second input is very small.
 
-* REPARTITION_HASH_FIRST: The system partitions (shuffles) each input (unless the input is already
+* `REPARTITION_HASH_FIRST`: The system partitions (shuffles) each input (unless the input is already
   partitioned) and builds a hash table from the first input. This strategy is good if the first
   input is smaller than the second, but both inputs are still large.
   *Note:* This is the default fallback strategy that the system uses if no size estimates can be made
   and no pre-existing partitiongs and sort-orders can be re-used.
 
-* REPARTITION_HASH_SECOND: The system partitions (shuffles) each input (unless the input is already
+* `REPARTITION_HASH_SECOND`: The system partitions (shuffles) each input (unless the input is already
   partitioned) and builds a hash table from the second input. This strategy is good if the second
   input is smaller than the first, but both inputs are still large.
 
-* REPARTITION_SORT_MERGE: The system partitions (shuffles) each input (unless the input is already
+* `REPARTITION_SORT_MERGE`: The system partitions (shuffles) each input (unless the input is already
   partitioned) and sorts each input (unless it is already sorted). The inputs are joined by
   a streamed merge of the sorted inputs. This strategy is good if one or both of the inputs are
   already sorted.
@@ -1558,9 +1558,13 @@ to manually pick a strategy, in case you want to enforce a specific way of execu
 DataSet<SomeType> input1 = // [...]
 DataSet<AnotherType> input2 = // [...]
 
-DataSet<Tuple2<SomeType, AnotherType> result =
+DataSet<Tuple2<SomeType, AnotherType> result1 =
       input1.leftOuterJoin(input2, JoinHint.REPARTITION_SORT_MERGE)
             .where("id").equalTo("key");
+
+DataSet<Tuple2<SomeType, AnotherType> result2 =
+      input1.rightOuterJoin(input2, JoinHint.BROADCAST_HASH_FIRST)
+            .where("id").equalTo("key");
 ~~~
 
 </div>
@@ -1573,6 +1577,8 @@ val input2: DataSet[AnotherType] = // [...]
 // hint that the second DataSet is very small
 val result1 = input1.leftOuterJoin(input2, JoinHint.REPARTITION_SORT_MERGE).where("id").equalTo("key")
 
+val result2 = input1.rightOuterJoin(input2, JoinHint.BROADCAST_HASH_FIRST).where("id").equalTo("key")
+
 ~~~
 
 </div>
@@ -1585,15 +1591,47 @@ Not supported.
 </div>
 </div>
 
-**NOTE:** Right now, outer joins can only be executed using the `REPARTITION_SORT_MERGE` strategy. Further execution strategies will be added in the future.
+The following hints are available.
 
-* OPTIMIZER_CHOOSES: Equivalent to not giving a hint at all, leaves the choice to the system.
+* `OPTIMIZER_CHOOSES`: Equivalent to not giving a hint at all, leaves the choice to the system.
 
-* REPARTITION_SORT_MERGE: The system partitions (shuffles) each input (unless the input is already
+* `BROADCAST_HASH_FIRST`: Broadcasts the first input and builds a hash table from it, which is
+  probed by the second input. A good strategy if the first input is very small.
+
+* `BROADCAST_HASH_SECOND`: Broadcasts the second input and builds a hash table from it, which is
+  probed by the first input. A good strategy if the second input is very small.
+
+* `REPARTITION_HASH_FIRST`: The system partitions (shuffles) each input (unless the input is already
+  partitioned) and builds a hash table from the first input. This strategy is good if the first
+  input is smaller than the second, but both inputs are still large.
+  
+* `REPARTITION_HASH_SECOND`: The system partitions (shuffles) each input (unless the input is already
+  partitioned) and builds a hash table from the second input. This strategy is good if the second
+  input is smaller than the first, but both inputs are still large.
+
+* `REPARTITION_SORT_MERGE`: The system partitions (shuffles) each input (unless the input is already
   partitioned) and sorts each input (unless it is already sorted). The inputs are joined by
   a streamed merge of the sorted inputs. This strategy is good if one or both of the inputs are
   already sorted.
 
+**NOTE:** Not all execution strategies are supported by every outer join type, yet.
+
+* `LeftOuterJoin` supports:
+  * `OPTIMIZER_CHOOSES`
+  * `BROADCAST_HASH_SECOND`
+  * `REPARTITION_HASH_SECOND`
+  * `REPARTITION_SORT_MERGE`
+
+* `RightOuterJoin` supports:
+  * `OPTIMIZER_CHOOSES`
+  * `BROADCAST_HASH_FIRST`
+  * `REPARTITION_HASH_FIRST`
+  * `REPARTITION_SORT_MERGE`
+
+* `FullOuterJoin` supports:
+  * `OPTIMIZER_CHOOSES`
+  * `REPARTITION_SORT_MERGE`
+
 
 ### Cross
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 164b4af..3101f35 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
@@ -841,7 +841,16 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> JoinOperatorSetsBase<T, R> leftOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.LEFT_OUTER);
+		switch(strategy) {
+			case OPTIMIZER_CHOOSES:
+			case REPARTITION_SORT_MERGE:
+			case REPARTITION_HASH_SECOND:
+			case BROADCAST_HASH_SECOND:
+				return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.LEFT_OUTER);
+			default:
+				throw new InvalidProgramException("Invalid JoinHint for LeftOuterJoin: "+strategy);
+		}
+
 	}
 
 	/**
@@ -881,7 +890,15 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> JoinOperatorSetsBase<T, R> rightOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.RIGHT_OUTER);
+		switch(strategy) {
+			case OPTIMIZER_CHOOSES:
+			case REPARTITION_SORT_MERGE:
+			case REPARTITION_HASH_FIRST:
+			case BROADCAST_HASH_FIRST:
+				return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.RIGHT_OUTER);
+			default:
+			throw new InvalidProgramException("Invalid JoinHint for RightOuterJoin: "+strategy);
+		}
 	}
 
 	/**
@@ -921,7 +938,13 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> JoinOperatorSetsBase<T, R> fullOuterJoin(DataSet<R> other, JoinHint strategy) {
-		return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.FULL_OUTER);
+		switch(strategy) {
+			case OPTIMIZER_CHOOSES:
+			case REPARTITION_SORT_MERGE:
+				return new JoinOperatorSetsBase<>(this, other, strategy, JoinType.FULL_OUTER);
+			default:
+			throw new InvalidProgramException("Invalid JoinHint for FullOuterJoin: "+strategy);
+		}
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
new file mode 100644
index 0000000..fe362f0
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
@@ -0,0 +1,250 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.api.common.InvalidProgramException;
+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.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.CompositeType;
+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.Tuple5;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class FullOuterJoinOperatorTest {
+
+	// TUPLE DATA
+	private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+		new ArrayList<>();
+
+	private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+		TupleTypeInfo<>(
+		BasicTypeInfo.INT_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.INT_TYPE_INFO
+	);
+
+	@Test
+	public void testFullOuter1() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+			.where(0).equalTo(4)
+			.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuter2() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+				.where("f1").equalTo("f3")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuter3() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuter4() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+				.where(0).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuter5() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo("f4")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuter6() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2)
+				.where("f0").equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testFullOuter7() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key position
+		ds1.fullOuterJoin(ds2)
+				.where(5).equalTo(0)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = CompositeType.InvalidFieldReferenceException.class)
+	public void testFullOuter8() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key reference
+		ds1.fullOuterJoin(ds2)
+				.where(1).equalTo("f5")
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuter9() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.fullOuterJoin(ds2)
+				.where(0).equalTo(1)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuter10() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.fullOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new LongKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testFullOuterStrategy1() {
+		this.testFullOuterStrategies(JoinHint.OPTIMIZER_CHOOSES);
+	}
+
+	@Test
+	public void testFullOuterStrategy2() {
+		this.testFullOuterStrategies(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuterStrategy3() {
+		this.testFullOuterStrategies(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuterStrategy4() {
+		this.testFullOuterStrategies(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuterStrategy5() {
+		this.testFullOuterStrategies(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testFullOuterStrategy6() {
+		this.testFullOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+
+	private void testFullOuterStrategies(JoinHint hint) {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.fullOuterJoin(ds2, hint)
+				.where(0).equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	
+	/*
+	 * ####################################################################
+	 */
+
+	@SuppressWarnings("serial")
+	public static class DummyJoin implements
+			JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long join(Tuple5<Integer, Long, String, Long, Integer> v1, Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+			return 1L;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+
+		@Override
+		public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f0;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f1;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
new file mode 100644
index 0000000..06b0c13
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
@@ -0,0 +1,251 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.typeutils.CompositeType;
+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.Tuple5;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class LeftOuterJoinOperatorTest {
+
+	// TUPLE DATA
+	private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+		new ArrayList<>();
+
+	private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+		TupleTypeInfo<>(
+		BasicTypeInfo.INT_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.INT_TYPE_INFO
+	);
+
+	@Test
+	public void testLeftOuter1() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+			.where(0).equalTo(4)
+			.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuter2() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+				.where("f1").equalTo("f3")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuter3() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuter4() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+				.where(0).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuter5() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo("f4")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuter6() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2)
+				.where("f0").equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testLeftOuter7() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key position
+		ds1.leftOuterJoin(ds2)
+				.where(5).equalTo(0)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = CompositeType.InvalidFieldReferenceException.class)
+	public void testLeftOuter8() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key reference
+		ds1.leftOuterJoin(ds2)
+				.where(1).equalTo("f5")
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testLeftOuter9() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.leftOuterJoin(ds2)
+				.where(0).equalTo(1)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testLeftOuter10() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.leftOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new LongKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testLeftOuterStrategy1() {
+		this.testLeftOuterStrategies(JoinHint.OPTIMIZER_CHOOSES);
+	}
+
+	@Test
+	public void testLeftOuterStrategy2() {
+		this.testLeftOuterStrategies(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testLeftOuterStrategy3() {
+		this.testLeftOuterStrategies(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test
+	public void testLeftOuterStrategy4() {
+		this.testLeftOuterStrategies(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testLeftOuterStrategy5() {
+		this.testLeftOuterStrategies(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testLeftOuterStrategy6() {
+		this.testLeftOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+
+	private void testLeftOuterStrategies(JoinHint hint) {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.leftOuterJoin(ds2, hint)
+				.where(0).equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	
+	/*
+	 * ####################################################################
+	 */
+
+	@SuppressWarnings("serial")
+	public static class DummyJoin implements
+			JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long join(Tuple5<Integer, Long, String, Long, Integer> v1, Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+			return 1L;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+
+		@Override
+		public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f0;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f1;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
new file mode 100644
index 0000000..0e407ca
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
@@ -0,0 +1,250 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.api.common.InvalidProgramException;
+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.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.CompositeType;
+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.Tuple5;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class RightOuterJoinOperatorTest {
+
+	// TUPLE DATA
+	private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+		new ArrayList<>();
+
+	private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+		TupleTypeInfo<>(
+		BasicTypeInfo.INT_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.INT_TYPE_INFO
+	);
+
+	@Test
+	public void testRightOuter1() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+			.where(0).equalTo(4)
+			.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuter2() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+				.where("f1").equalTo("f3")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuter3() {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuter4() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+				.where(0).equalTo(new IntKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuter5() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo("f4")
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuter6() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2)
+				.where("f0").equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testRightOuter7() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key position
+		ds1.rightOuterJoin(ds2)
+				.where(5).equalTo(0)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = CompositeType.InvalidFieldReferenceException.class)
+	public void testRightOuter8() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// invalid key reference
+		ds1.rightOuterJoin(ds2)
+				.where(1).equalTo("f5")
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testRightOuter9() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.rightOuterJoin(ds2)
+				.where(0).equalTo(1)
+				.with(new DummyJoin());
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testRightOuter10() {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// key types do not match
+		ds1.rightOuterJoin(ds2)
+				.where(new IntKeySelector()).equalTo(new LongKeySelector())
+				.with(new DummyJoin());
+	}
+
+	@Test
+	public void testRightOuterStrategy1() {
+		this.testRightOuterStrategies(JoinHint.OPTIMIZER_CHOOSES);
+	}
+
+	@Test
+	public void testRightOuterStrategy2() {
+		this.testRightOuterStrategies(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testRightOuterStrategy3() {
+		this.testRightOuterStrategies(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testRightOuterStrategy4() {
+		this.testRightOuterStrategies(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	@Test
+	public void testRightOuterStrategy5() {
+		this.testRightOuterStrategies(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test
+	public void testRightOuterStrategy6() {
+		this.testRightOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+
+	private void testRightOuterStrategies(JoinHint hint) {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		ds1.rightOuterJoin(ds2, hint)
+				.where(0).equalTo(4)
+				.with(new DummyJoin());
+	}
+
+	
+	/*
+	 * ####################################################################
+	 */
+
+	@SuppressWarnings("serial")
+	public static class DummyJoin implements
+			JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long join(Tuple5<Integer, Long, String, Long, Integer> v1, Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+			return 1L;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+
+		@Override
+		public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f0;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+
+		@Override
+		public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
+			return v.f1;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 553c127..5ff9eaf 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
@@ -202,9 +202,11 @@ public abstract class CostEstimator {
 			addLocalMergeCost(firstInput, secondInput, driverCosts, costWeight);
 			break;
 		case HYBRIDHASH_BUILD_FIRST:
+		case RIGHT_HYBRIDHASH_BUILD_FIRST:
 			addHybridHashCosts(firstInput, secondInput, driverCosts, costWeight);
 			break;
 		case HYBRIDHASH_BUILD_SECOND:
+		case LEFT_HYBRIDHASH_BUILD_SECOND:
 			addHybridHashCosts(secondInput, firstInput, driverCosts, costWeight);
 			break;
 		case HYBRIDHASH_BUILD_FIRST_CACHED:

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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
index ebdfcc8..0784de3 100644
--- 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
@@ -25,6 +25,8 @@ import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoi
 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.HashLeftOuterJoinBuildSecondDescriptor;
+import org.apache.flink.optimizer.operators.HashRightOuterJoinBuildFirstDescriptor;
 import org.apache.flink.optimizer.operators.OperatorDescriptorDual;
 import org.apache.flink.optimizer.operators.SortMergeFullOuterJoinDescriptor;
 import org.apache.flink.optimizer.operators.SortMergeLeftOuterJoinDescriptor;
@@ -56,20 +58,19 @@ public class OuterJoinNode extends TwoInputNode {
 		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));
+		List<OperatorDescriptorDual> list;
+		switch (type) {
+			case LEFT:
+				list = createLeftOuterJoinDescriptors(joinHint);
 				break;
-			case REPARTITION_SORT_MERGE:
-				list.add(getSortMergeDescriptor(type, false));
+			case RIGHT:
+				list = createRightOuterJoinDescriptors(joinHint);
+				break;
+			case FULL:
+				list = createFullOuterJoinDescriptors(joinHint);
 				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);
+				throw new CompilerException("Unknown outer join type: " + type);
 		}
 
 		Partitioner<?> customPartitioner = operator.getCustomPartitioner();
@@ -81,14 +82,74 @@ public class OuterJoinNode extends TwoInputNode {
 		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);
+	private List<OperatorDescriptorDual> createLeftOuterJoinDescriptors(JoinHint hint) {
+
+		List<OperatorDescriptorDual> list = new ArrayList<>();
+		switch (hint) {
+			case OPTIMIZER_CHOOSES:
+				list.add(new SortMergeLeftOuterJoinDescriptor(this.keys1, this.keys2, true));
+				list.add(new HashLeftOuterJoinBuildSecondDescriptor(this.keys1, this.keys2, true, true));
+				break;
+			case REPARTITION_SORT_MERGE:
+				list.add(new SortMergeLeftOuterJoinDescriptor(this.keys1, this.keys2, false));
+				break;
+			case REPARTITION_HASH_SECOND:
+				list.add(new HashLeftOuterJoinBuildSecondDescriptor(this.keys1, this.keys2, false, true));
+				break;
+			case BROADCAST_HASH_SECOND:
+				list.add(new HashLeftOuterJoinBuildSecondDescriptor(this.keys1, this.keys2, true, false));
+				break;
+			case BROADCAST_HASH_FIRST:
+			case REPARTITION_HASH_FIRST:
+			default:
+				throw new CompilerException("Invalid join hint: " + hint + " for left outer join");
 		}
+		return list;
+	}
+
+	private List<OperatorDescriptorDual> createRightOuterJoinDescriptors(JoinHint hint) {
+
+		List<OperatorDescriptorDual> list = new ArrayList<>();
+		switch (hint) {
+			case OPTIMIZER_CHOOSES:
+				list.add(new SortMergeRightOuterJoinDescriptor(this.keys1, this.keys2, true));
+				list.add(new HashRightOuterJoinBuildFirstDescriptor(this.keys1, this.keys2, true, true));
+				break;
+			case REPARTITION_SORT_MERGE:
+				list.add(new SortMergeRightOuterJoinDescriptor(this.keys1, this.keys2, false));
+				break;
+			case REPARTITION_HASH_FIRST:
+				list.add(new HashRightOuterJoinBuildFirstDescriptor(this.keys1, this.keys2, false, true));
+				break;
+			case BROADCAST_HASH_FIRST:
+				list.add(new HashRightOuterJoinBuildFirstDescriptor(this.keys1, this.keys2, true, false));
+				break;
+			case BROADCAST_HASH_SECOND:
+			case REPARTITION_HASH_SECOND:
+			default:
+				throw new CompilerException("Invalid join hint: " + hint + " for right outer join");
+		}
+		return list;
+	}
+
+	private List<OperatorDescriptorDual> createFullOuterJoinDescriptors(JoinHint hint) {
+
+		List<OperatorDescriptorDual> list = new ArrayList<>();
+		switch (hint) {
+			case OPTIMIZER_CHOOSES:
+				list.add(new SortMergeFullOuterJoinDescriptor(this.keys1, this.keys2));
+				break;
+			case REPARTITION_SORT_MERGE:
+				list.add(new SortMergeFullOuterJoinDescriptor(this.keys1, this.keys2));
+				break;
+			case REPARTITION_HASH_SECOND:
+			case BROADCAST_HASH_SECOND:
+			case BROADCAST_HASH_FIRST:
+			case REPARTITION_HASH_FIRST:
+			default:
+				throw new CompilerException("Invalid join hint: " + hint + " for full outer join");
+		}
+		return list;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java
new file mode 100644
index 0000000..8ed7969
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+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.runtime.operators.DriverStrategy;
+
+import java.util.Collections;
+import java.util.List;
+
+public class HashLeftOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor {
+
+	public HashLeftOuterJoinBuildSecondDescriptor(FieldList keys1, FieldList keys2,
+													boolean broadcastSecondAllowed, boolean repartitionAllowed) {
+		super(keys1, keys2, false, broadcastSecondAllowed, repartitionAllowed);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND;
+	}
+
+	@Override
+	protected List<LocalPropertiesPair> createPossibleLocalProperties() {
+		// all properties are possible
+		return Collections.singletonList(new LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties()));
+	}
+	
+	@Override
+	public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2,
+			LocalProperties produced1, LocalProperties produced2) {
+		return true;
+	}
+
+	@Override
+	public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) {
+
+		String nodeName = "LeftOuterJoin("+node.getOperator().getName()+")";
+		return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2);
+	}
+	
+	@Override
+	public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) {
+		return new LocalProperties();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java
new file mode 100644
index 0000000..5ddba1c
--- /dev/null
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.optimizer.operators;
+
+import org.apache.flink.api.common.operators.util.FieldList;
+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.runtime.operators.DriverStrategy;
+
+import java.util.Collections;
+import java.util.List;
+
+public class HashRightOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor {
+
+	public HashRightOuterJoinBuildFirstDescriptor(FieldList keys1, FieldList keys2,
+													boolean broadcastFirstAllowed, boolean repartitionAllowed) {
+		super(keys1, keys2, broadcastFirstAllowed, false, repartitionAllowed);
+	}
+
+	@Override
+	public DriverStrategy getStrategy() {
+		return DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST;
+	}
+
+	@Override
+	protected List<LocalPropertiesPair> createPossibleLocalProperties() {
+		// all properties are possible
+		return Collections.singletonList(new LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties()));
+	}
+	
+	@Override
+	public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2,
+			LocalProperties produced1, LocalProperties produced2) {
+		return true;
+	}
+
+	@Override
+	public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) {
+
+		String nodeName = "RightOuterJoin("+node.getOperator().getName()+")";
+		return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2);
+	}
+	
+	@Override
+	public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) {
+		return new LocalProperties();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
index 8f72754..9ac2ed6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
@@ -25,10 +25,10 @@ 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.configuration.ConfigConstants;
-import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstReOpenableHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondReOpenableHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.ReusingBuildFirstReOpenableHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.ReusingBuildSecondReOpenableHashMatchIterator;
+import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstReOpenableHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondReOpenableHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildFirstReOpenableHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildSecondReOpenableHashJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;
@@ -85,7 +85,7 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 		if (objectReuseEnabled) {
 			if (buildSideIndex == 0 && probeSideIndex == 1) {
 
-				matchIterator = new ReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT>(
+				matchIterator = new ReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT>(
 						input1, input2,
 						serializer1, comparator1,
 						serializer2, comparator2,
@@ -94,12 +94,13 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 						this.taskContext.getIOManager(),
 						this.taskContext.getOwningNepheleTask(),
 						availableMemory,
+						false,
 						hashJoinUseBitMaps);
 
 
 			} else if (buildSideIndex == 1 && probeSideIndex == 0) {
 
-				matchIterator = new ReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT>(
+				matchIterator = new ReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT>(
 						input1, input2,
 						serializer1, comparator1,
 						serializer2, comparator2,
@@ -108,6 +109,7 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 						this.taskContext.getIOManager(),
 						this.taskContext.getOwningNepheleTask(),
 						availableMemory,
+						false,
 						hashJoinUseBitMaps);
 
 			} else {
@@ -116,7 +118,7 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 		} else {
 			if (buildSideIndex == 0 && probeSideIndex == 1) {
 
-				matchIterator = new NonReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT>(
+				matchIterator = new NonReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT>(
 						input1, input2,
 						serializer1, comparator1,
 						serializer2, comparator2,
@@ -125,12 +127,13 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 						this.taskContext.getIOManager(),
 						this.taskContext.getOwningNepheleTask(),
 						availableMemory,
+						false,
 						hashJoinUseBitMaps);
 
 
 			} else if (buildSideIndex == 1 && probeSideIndex == 0) {
 
-				matchIterator = new NonReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT>(
+				matchIterator = new NonReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT>(
 						input1, input2,
 						serializer1, comparator1,
 						serializer2, comparator2,
@@ -139,6 +142,7 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 						this.taskContext.getIOManager(),
 						this.taskContext.getOwningNepheleTask(),
 						availableMemory,
+						false,
 						hashJoinUseBitMaps);
 
 			} else {
@@ -173,20 +177,20 @@ public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends Jo
 
 		if (objectReuseEnabled) {
 			if (buildSideIndex == 0 && probeSideIndex == 1) {
-				final ReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT> matchIterator = (ReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT>) this.matchIterator;
+				final ReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT> matchIterator = (ReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT>) this.matchIterator;
 
 				matchIterator.reopenProbe(input2);
 			} else {
-				final ReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT> matchIterator = (ReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT>) this.matchIterator;
+				final ReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT> matchIterator = (ReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT>) this.matchIterator;
 				matchIterator.reopenProbe(input1);
 			}
 		} else {
 			if (buildSideIndex == 0 && probeSideIndex == 1) {
-				final NonReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT> matchIterator = (NonReusingBuildFirstReOpenableHashMatchIterator<IT1, IT2, OT>) this.matchIterator;
+				final NonReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT> matchIterator = (NonReusingBuildFirstReOpenableHashJoinIterator<IT1, IT2, OT>) this.matchIterator;
 
 				matchIterator.reopenProbe(input2);
 			} else {
-				final NonReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT> matchIterator = (NonReusingBuildSecondReOpenableHashMatchIterator<IT1, IT2, OT>) this.matchIterator;
+				final NonReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT> matchIterator = (NonReusingBuildSecondReOpenableHashJoinIterator<IT1, IT2, OT>) this.matchIterator;
 				matchIterator.reopenProbe(input1);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
index 8c964d4..2589ca5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
@@ -81,8 +81,7 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<Fl
 		final IOManager ioManager = this.taskContext.getIOManager();
 		
 		// set up memory and I/O parameters
-		final double fractionAvailableMemory = config.getRelativeMemoryDriver();
-		final int numPages = memoryManager.computeNumberOfPages(fractionAvailableMemory);
+		final double driverMemFraction = config.getRelativeMemoryDriver();
 		
 		final DriverStrategy ls = config.getDriverStrategy();
 		
@@ -121,7 +120,7 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<Fl
 					pairComparatorFactory,
 					memoryManager,
 					ioManager,
-					numPages
+					driverMemFraction
 			);
 		} else {
 			this.outerJoinIterator = getNonReusingOuterJoinIterator(
@@ -135,7 +134,7 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<Fl
 					pairComparatorFactory,
 					memoryManager,
 					ioManager,
-					numPages
+					driverMemFraction
 			);
 		}
 		
@@ -183,7 +182,7 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<Fl
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception;
 	
 	protected abstract JoinTaskIterator<IT1, IT2, OT> getNonReusingOuterJoinIterator(
@@ -197,6 +196,6 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<Fl
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
index bc7bee5..b069f12 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
@@ -73,11 +73,8 @@ public enum DriverStrategy {
 
 	// both inputs are merged, but materialized to the side for block-nested-loop-join among values with equal key
 	INNER_MERGE(JoinDriver.class, null, MATERIALIZING, MATERIALIZING, 2),
-
 	LEFT_OUTER_MERGE(LeftOuterJoinDriver.class, null, MATERIALIZING, MATERIALIZING, 2),
-
 	RIGHT_OUTER_MERGE(RightOuterJoinDriver.class, null, MATERIALIZING, MATERIALIZING, 2),
-
 	FULL_OUTER_MERGE(FullOuterJoinDriver.class, null, MATERIALIZING, MATERIALIZING, 2),
 
 	// co-grouping inputs
@@ -94,6 +91,11 @@ public enum DriverStrategy {
 	HYBRIDHASH_BUILD_FIRST_CACHED(BuildFirstCachedJoinDriver.class, null, FULL_DAM, MATERIALIZING, 2),
 	//  cached variant of HYBRIDHASH_BUILD_SECOND, that can only be used inside of iterations
 	HYBRIDHASH_BUILD_SECOND_CACHED(BuildSecondCachedJoinDriver.class, null, MATERIALIZING, FULL_DAM, 2),
+
+	// right outer join, the first input is build side, the second side is probe side of a hybrid hash table
+	RIGHT_HYBRIDHASH_BUILD_FIRST(RightOuterJoinDriver.class, null, FULL_DAM, MATERIALIZING, 2),
+	// left outer join, the second input is build side, the first side is probe side of a hybrid hash table
+	LEFT_HYBRIDHASH_BUILD_SECOND(LeftOuterJoinDriver.class, null, MATERIALIZING, FULL_DAM, 2),
 	
 	// the second input is inner loop, the first input is outer loop and block-wise processed
 	NESTEDLOOP_BLOCKED_OUTER_FIRST(CrossDriver.class, null, MATERIALIZING, FULL_DAM, 0),

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 d942b72..2c01fec 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
@@ -47,10 +47,11 @@ public class FullOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case FULL_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new ReusingMergeOuterJoinIterator<>(
 						OuterJoinType.FULL,
 						in1,
@@ -82,10 +83,11 @@ public class FullOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case FULL_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new NonReusingMergeOuterJoinIterator<>(
 						OuterJoinType.FULL,
 						in1,

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
index 7a9c8e6..c55843a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
@@ -26,10 +26,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator;
-import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator;
+import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashJoinIterator;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeInnerJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
@@ -133,23 +133,25 @@ public class JoinDriver<IT1, IT2, OT> implements Driver<FlatJoinFunction<IT1, IT
 							memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask());
 					break;
 				case HYBRIDHASH_BUILD_FIRST:
-					this.joinIterator = new ReusingBuildFirstHashMatchIterator<>(in1, in2,
+					this.joinIterator = new ReusingBuildFirstHashJoinIterator<>(in1, in2,
 							serializer1, comparator1,
 							serializer2, comparator2,
 							pairComparatorFactory.createComparator21(comparator1, comparator2),
 							memoryManager, ioManager,
 							this.taskContext.getOwningNepheleTask(),
 							fractionAvailableMemory,
+							false,
 							hashJoinUseBitMaps);
 					break;
 				case HYBRIDHASH_BUILD_SECOND:
-					this.joinIterator = new ReusingBuildSecondHashMatchIterator<>(in1, in2,
+					this.joinIterator = new ReusingBuildSecondHashJoinIterator<>(in1, in2,
 							serializer1, comparator1,
 							serializer2, comparator2,
 							pairComparatorFactory.createComparator12(comparator1, comparator2),
 							memoryManager, ioManager,
 							this.taskContext.getOwningNepheleTask(),
 							fractionAvailableMemory,
+							false,
 							hashJoinUseBitMaps);
 					break;
 				default:
@@ -166,23 +168,25 @@ public class JoinDriver<IT1, IT2, OT> implements Driver<FlatJoinFunction<IT1, IT
 
 					break;
 				case HYBRIDHASH_BUILD_FIRST:
-					this.joinIterator = new NonReusingBuildFirstHashMatchIterator<>(in1, in2,
+					this.joinIterator = new NonReusingBuildFirstHashJoinIterator<>(in1, in2,
 							serializer1, comparator1,
 							serializer2, comparator2,
 							pairComparatorFactory.createComparator21(comparator1, comparator2),
 							memoryManager, ioManager,
 							this.taskContext.getOwningNepheleTask(),
 							fractionAvailableMemory,
+							false,
 							hashJoinUseBitMaps);
 					break;
 				case HYBRIDHASH_BUILD_SECOND:
-					this.joinIterator = new NonReusingBuildSecondHashMatchIterator<>(in1, in2,
+					this.joinIterator = new NonReusingBuildSecondHashJoinIterator<>(in1, in2,
 							serializer1, comparator1,
 							serializer2, comparator2,
 							pairComparatorFactory.createComparator12(comparator1, comparator2),
 							memoryManager, ioManager,
 							this.taskContext.getOwningNepheleTask(),
 							fractionAvailableMemory,
+							false,
 							hashJoinUseBitMaps);
 					break;
 				default:

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 ae05d1e..49d3648 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
@@ -24,6 +24,8 @@ 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.hash.NonReusingBuildSecondHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashJoinIterator;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
@@ -47,10 +49,11 @@ public class LeftOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case LEFT_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new ReusingMergeOuterJoinIterator<>(
 						OuterJoinType.LEFT,
 						in1,
@@ -65,6 +68,16 @@ public class LeftOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 						numPages,
 						super.taskContext.getOwningNepheleTask()
 				);
+			case LEFT_HYBRIDHASH_BUILD_SECOND:
+				return new ReusingBuildSecondHashJoinIterator<>(in1, in2,
+						serializer1, comparator1,
+						serializer2, comparator2,
+						pairComparatorFactory.createComparator12(comparator1, comparator2),
+						memoryManager, ioManager,
+						this.taskContext.getOwningNepheleTask(),
+						driverMemFraction,
+						true,
+						false);
 			default:
 				throw new Exception("Unsupported driver strategy for left outer join driver: " + driverStrategy.name());
 		}
@@ -82,10 +95,11 @@ public class LeftOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case LEFT_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new NonReusingMergeOuterJoinIterator<>(
 						OuterJoinType.LEFT,
 						in1,
@@ -100,6 +114,16 @@ public class LeftOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<I
 						numPages,
 						super.taskContext.getOwningNepheleTask()
 				);
+			case LEFT_HYBRIDHASH_BUILD_SECOND:
+				return new NonReusingBuildSecondHashJoinIterator<>(in1, in2,
+						serializer1, comparator1,
+						serializer2, comparator2,
+						pairComparatorFactory.createComparator12(comparator1, comparator2),
+						memoryManager, ioManager,
+						this.taskContext.getOwningNepheleTask(),
+						driverMemFraction,
+						true,
+						false);
 			default:
 				throw new Exception("Unsupported driver strategy for left outer join driver: " + driverStrategy.name());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/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 6fc8abd..1b67397 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
@@ -24,6 +24,8 @@ 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.hash.NonReusingBuildFirstHashJoinIterator;
+import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashJoinIterator;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
@@ -47,10 +49,11 @@ public class RightOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case RIGHT_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new ReusingMergeOuterJoinIterator<>(
 						OuterJoinType.RIGHT,
 						in1,
@@ -65,6 +68,16 @@ public class RightOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<
 						numPages,
 						super.taskContext.getOwningNepheleTask()
 				);
+			case RIGHT_HYBRIDHASH_BUILD_FIRST:
+				return new ReusingBuildFirstHashJoinIterator<>(in1, in2,
+						serializer1, comparator1,
+						serializer2, comparator2,
+						pairComparatorFactory.createComparator21(comparator1, comparator2),
+						memoryManager, ioManager,
+						this.taskContext.getOwningNepheleTask(),
+						driverMemFraction,
+						true,
+						false);
 			default:
 				throw new Exception("Unsupported driver strategy for right outer join driver: " + driverStrategy.name());
 		}
@@ -82,10 +95,11 @@ public class RightOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<
 			TypePairComparatorFactory<IT1, IT2> pairComparatorFactory,
 			MemoryManager memoryManager,
 			IOManager ioManager,
-			int numPages
+			double driverMemFraction
 	) throws Exception {
 		switch (driverStrategy) {
 			case RIGHT_OUTER_MERGE:
+				int numPages = memoryManager.computeNumberOfPages(driverMemFraction);
 				return new NonReusingMergeOuterJoinIterator<>(
 						OuterJoinType.RIGHT,
 						in1,
@@ -100,6 +114,16 @@ public class RightOuterJoinDriver<IT1, IT2, OT> extends AbstractOuterJoinDriver<
 						numPages,
 						super.taskContext.getOwningNepheleTask()
 				);
+			case RIGHT_HYBRIDHASH_BUILD_FIRST:
+				return new NonReusingBuildFirstHashJoinIterator<>(in1, in2,
+						serializer1, comparator1,
+						serializer2, comparator2,
+						pairComparatorFactory.createComparator21(comparator1, comparator2),
+						memoryManager, ioManager,
+						this.taskContext.getOwningNepheleTask(),
+						driverMemFraction,
+						true,
+						false);
 			default:
 				throw new Exception("Unsupported driver strategy for right outer join driver: " + driverStrategy.name());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashJoinIteratorBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashJoinIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashJoinIteratorBase.java
new file mode 100644
index 0000000..30b1df2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashJoinIteratorBase.java
@@ -0,0 +1,56 @@
+/*
+ * 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.runtime.operators.hash;
+
+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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+
+import java.util.List;
+
+/**
+ * Common methods for all Hash Join Iterators.
+ */
+public class HashJoinIteratorBase {
+	
+	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
+			TypeSerializer<BT> buildSideSerializer,
+			TypeComparator<BT> buildSideComparator,
+			TypeSerializer<PT> probeSideSerializer,
+			TypeComparator<PT> probeSideComparator,
+			TypePairComparator<PT, BT> pairComparator,
+			MemoryManager memManager,
+			IOManager ioManager,
+			AbstractInvokable ownerTask,
+			double memoryFraction,
+			boolean useBloomFilters) throws MemoryAllocationException {
+
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
+		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
+		
+		return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
+				buildSideComparator, probeSideComparator, pairComparator,
+				memorySegments, ioManager,
+				useBloomFilters);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
deleted file mode 100644
index 3b12c68..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
+++ /dev/null
@@ -1,56 +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.runtime.operators.hash;
-
-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.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-
-import java.util.List;
-
-/**
- * Common methods for all Hash Join Iterators.
- */
-public class HashMatchIteratorBase {
-	
-	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(
-			TypeSerializer<BT> buildSideSerializer,
-			TypeComparator<BT> buildSideComparator,
-			TypeSerializer<PT> probeSideSerializer,
-			TypeComparator<PT> probeSideComparator,
-			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager,
-			IOManager ioManager,
-			AbstractInvokable ownerTask,
-			double memoryFraction,
-			boolean useBloomFilters) throws MemoryAllocationException {
-
-		final int numPages = memManager.computeNumberOfPages(memoryFraction);
-		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
-		
-		return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer,
-				buildSideComparator, probeSideComparator, pairComparator,
-				memorySegments, ioManager,
-				useBloomFilters);
-	}
-}


[03/10] flink git commit: [FLINK-2857] [gelly] Improve Gelly API and documentation. - Improve javadocs of Graph creation methods - Add fromTuple2 creation methods - Rename mapper parameters to vertexInitializer. - Improve javadocs and parameter names of

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
index b7e3385..2a10bd1 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeJoinFunction;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Triplet;
 import org.apache.flink.graph.Vertex;
@@ -90,11 +91,10 @@ public class EuclideanGraphWeighing implements ProgramDescription {
 				});
 
 		Graph<Long, Point, Double> resultedGraph = graph.joinWithEdges(edgesWithEuclideanWeight,
-				new MapFunction<Tuple2<Double, Double>, Double>() {
+				new EdgeJoinFunction<Double, Double>() {
 
-					@Override
-					public Double map(Tuple2<Double, Double> distance) throws Exception {
-						return distance.f1;
+					public Double edgeJoin(Double edgeValue, Double inputValue) {
+						return inputValue;
 					}
 				});
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
index 0f84dbb..5fb75e2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
@@ -29,6 +29,7 @@ import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.ReduceNeighborsFunction;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.Triplet;
+import org.apache.flink.graph.VertexJoinFunction;
 import org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData;
 
 import java.util.HashSet;
@@ -88,12 +89,11 @@ public class JaccardSimilarityMeasure implements ProgramDescription {
 
 		// join with the vertices to update the node values
 		Graph<Long, HashSet<Long>, Double> graphWithVertexValues =
-				graph.joinWithVertices(computedNeighbors, new MapFunction<Tuple2<HashSet<Long>, HashSet<Long>>,
+				graph.joinWithVertices(computedNeighbors, new VertexJoinFunction<HashSet<Long>,
 						HashSet<Long>>() {
 
-					@Override
-					public HashSet<Long> map(Tuple2<HashSet<Long>, HashSet<Long>> tuple2) throws Exception {
-						return tuple2.f1;
+					public HashSet<Long> vertexJoin(HashSet<Long> vertexValue, HashSet<Long> inputValue) {
+						return inputValue;
 					}
 				});
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
index e347bc5..297dce2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
@@ -37,6 +37,7 @@ import org.apache.flink.graph.EdgeDirection;
 import org.apache.flink.graph.EdgesFunctionWithVertexValue;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.VertexJoinFunction;
 import org.apache.flink.graph.example.utils.MusicProfilesData;
 import org.apache.flink.graph.library.LabelPropagation;
 import org.apache.flink.types.NullValue;
@@ -149,9 +150,9 @@ public class MusicProfiles implements ProgramDescription {
 
 		DataSet<Vertex<String, Long>> verticesWithCommunity = similarUsersGraph
 				.joinWithVertices(idsWithInitialLabels,
-						new MapFunction<Tuple2<Long, Long>, Long>() {
-							public Long map(Tuple2<Long, Long> value) {
-								return value.f1;
+						new VertexJoinFunction<Long, Long>() {
+							public Long vertexJoin(Long vertexValue, Long inputValue) {
+								return inputValue;
 							}
 						}).run(new LabelPropagation<String, NullValue>(maxIterations));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
index 1694205..43a5e5c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.graph.library;
 
-import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.EdgeJoinFunction;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -85,7 +85,7 @@ public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet
 		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
 
 		Graph<K, Double, Double> networkWithWeights = network
-				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
+				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeights());
 
 		return networkWithWeights.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(),
 				new UpdateRanks<K>(beta, numberOfVertices), maxIterations)
@@ -144,9 +144,10 @@ public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet
 	}
 
 	@SuppressWarnings("serial")
-	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
-		public Double map(Tuple2<Double, Long> value) {
-			return value.f0 / value.f1;
+	private static final class InitWeights implements EdgeJoinFunction<Double, Long> {
+
+		public Double edgeJoin(Double edgeValue, Long inputValue) {
+			return edgeValue / (double) inputValue;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
index 76d170d..1eafce2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
@@ -31,6 +31,7 @@ import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Triplet;
 import org.apache.flink.graph.EdgeDirection;
 import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.VertexJoinFunction;
 import org.apache.flink.types.NullValue;
 
 import java.util.TreeMap;
@@ -143,12 +144,13 @@ public class GSATriangleCount<K extends Comparable<K>, VV, EV> implements
 	}
 
 	@SuppressWarnings("serial")
-	private static final class AttachValues<K> implements MapFunction<Tuple2<TreeMap<K, Integer>,
-			TreeMap<K, Integer>>, TreeMap<K, Integer>> {
+	private static final class AttachValues<K> implements VertexJoinFunction<TreeMap<K, Integer>,
+			TreeMap<K, Integer>> {
 
 		@Override
-		public TreeMap<K, Integer> map(Tuple2<TreeMap<K, Integer>, TreeMap<K, Integer>> tuple2) throws Exception {
-			return tuple2.f1;
+		public TreeMap<K, Integer> vertexJoin(TreeMap<K, Integer> vertexValue,
+				TreeMap<K, Integer> inputValue) {
+			return inputValue;
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
index c0e22c4..dfbe14b 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.graph.library;
 
-import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeJoinFunction;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -86,7 +86,7 @@ public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Ve
 		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
 
 		Graph<K, Double, Double> networkWithWeights = network
-				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
+				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeights());
 
 		return networkWithWeights.runVertexCentricIteration(new VertexRankUpdater<K>(beta, numberOfVertices),
 				new RankMessenger<K>(numberOfVertices), maxIterations)
@@ -149,9 +149,10 @@ public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Ve
 	}
 
 	@SuppressWarnings("serial")
-	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
-		public Double map(Tuple2<Double, Long> value) {
-			return value.f0 / value.f1;
+	private static final class InitWeights implements EdgeJoinFunction<Double, Long> {
+
+		public Double edgeJoin(Double edgeValue, Long inputValue) {
+			return edgeValue / (double) inputValue;
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
index 22a5151..20f4454 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
@@ -151,6 +152,59 @@ public class GraphCreationITCase extends MultipleProgramsTestBase {
 		compareResultAsText(result, expectedResult);
 	}
 
+	@Test
+	public void testFromTuple2() throws Exception {
+		/*
+		 * Test graph creation with fromTuple2DataSet
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple2<Long, Long>> edges = TestGraphUtils.getLongLongTuple2Data(env);
+
+		Graph<Long, NullValue, NullValue> graph = Graph.fromTuple2DataSet(edges, env);
+
+        List<Vertex<Long, NullValue>> result = graph.getVertices().collect();
+        
+		expectedResult = "1,(null)\n" +
+					"2,(null)\n" +
+					"3,(null)\n" +
+					"4,(null)\n" +
+					"6,(null)\n" +
+					"10,(null)\n" +
+					"20,(null)\n" +
+					"30,(null)\n" +
+					"40,(null)\n" +
+					"60,(null)\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testFromTuple2WithMapper() throws Exception {
+		/*
+		 * Test graph creation with fromTuple2DataSet with vertex initializer
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple2<Long, Long>> edges = TestGraphUtils.getLongLongTuple2Data(env);
+
+		Graph<Long, String, NullValue> graph = Graph.fromTuple2DataSet(edges,
+				new BooMapper(), env);
+
+        List<Vertex<Long, String>> result = graph.getVertices().collect();
+        
+		expectedResult = "1,boo\n" +
+					"2,boo\n" +
+					"3,boo\n" +
+					"4,boo\n" +
+					"6,boo\n" +
+					"10,boo\n" +
+					"20,boo\n" +
+					"30,boo\n" +
+					"40,boo\n" +
+					"60,boo\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
 	@SuppressWarnings("serial")
 	private static final class AssignIdAsValueMapper implements MapFunction<Long, Long> {
 		public Long map(Long vertexId) {
@@ -171,4 +225,9 @@ public class GraphCreationITCase extends MultipleProgramsTestBase {
 			return dummyValue;
 		}
 	}
+
+	@SuppressWarnings("serial")
+	private static final class BooMapper implements MapFunction<Long, String> {
+		public String map(Long value) {	return "boo"; }
+	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
index e406ce2..e0bc35a 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeJoinFunction;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
@@ -462,9 +463,10 @@ public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
     }
 
 	@SuppressWarnings("serial")
-	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-		public Long map(Tuple2<Long, Long> tuple) throws Exception {
-			return tuple.f0 + tuple.f1;
+	private static final class AddValuesMapper implements EdgeJoinFunction<Long, Long> {
+
+		public Long edgeJoin(Long edgeValue, Long inputValue) throws Exception {
+			return edgeValue + inputValue;
 		}
 	}
 
@@ -477,29 +479,34 @@ public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
     }
 
 	@SuppressWarnings("serial")
-	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
-        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
-            if(tuple.f1) {
-                return tuple.f0 * 2;
+	private static final class DoubleIfTrueMapper implements EdgeJoinFunction<Long, Boolean> {
+
+		public Long edgeJoin(Long edgeValue, Boolean inputValue) {
+            if(inputValue) {
+                return edgeValue * 2;
             }
             else {
-                return tuple.f0;
+                return edgeValue;
             }
-        }
+		}
     }
 
 	@SuppressWarnings("serial")
-	private static final class DoubleValueMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-        public Long map(Tuple2<Long, Long> tuple) throws Exception {
-            return tuple.f1 * 2;
-        }
+	private static final class DoubleValueMapper implements EdgeJoinFunction<Long, Long> {
+ 
+		public Long edgeJoin(Long edgeValue, Long inputValue) {
+			return inputValue*2;
+		}
     }
 
 	@SuppressWarnings("serial")
-	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
-        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
-            return (long) tuple.f1.getIntField();
-        }
+	private static final class CustomValueMapper implements EdgeJoinFunction<
+	Long, DummyCustomParameterizedType<Float>> {
+
+		public Long edgeJoin(Long edgeValue,
+				DummyCustomParameterizedType<Float> inputValue) {
+			return (long) inputValue.getIntField();
+		}
     }
 
 	@SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
index 22a5535..7a25788 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.VertexJoinFunction;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
 import org.apache.flink.graph.utils.VertexToTuple2Map;
@@ -173,9 +174,10 @@ public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
     }
 
 	@SuppressWarnings("serial")
-	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-		public Long map(Tuple2<Long, Long> tuple) throws Exception {
-			return tuple.f0 + tuple.f1;
+	private static final class AddValuesMapper implements VertexJoinFunction<Long, Long> {
+
+		public Long vertexJoin(Long vertexValue, Long inputValue) {
+			return vertexValue + inputValue;
 		}
 	}
 
@@ -187,28 +189,32 @@ public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
     }
 
 	@SuppressWarnings("serial")
-	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
-        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
-            if(tuple.f1) {
-                return tuple.f0 * 2;
+	private static final class DoubleIfTrueMapper implements VertexJoinFunction<Long, Boolean> {
+
+		public Long vertexJoin(Long vertexValue, Boolean inputValue) {
+            if(inputValue) {
+                return vertexValue * 2;
             }
             else {
-                return tuple.f0;
+                return vertexValue;
             }
-        }
+		}
     }
 
 	@SuppressWarnings("serial")
-	private static final class ProjectSecondMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-        public Long map(Tuple2<Long, Long> tuple) throws Exception {
-            return tuple.f1;
-        }
+	private static final class ProjectSecondMapper implements VertexJoinFunction<Long, Long> {
+
+		public Long vertexJoin(Long vertexValue, Long inputValue) {
+			return inputValue;
+		}
     }
 
 	@SuppressWarnings("serial")
-	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
-        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
-            return (long) tuple.f1.getIntField();
-        }
+	private static final class CustomValueMapper implements VertexJoinFunction<Long,
+		DummyCustomParameterizedType<Float>> {
+
+		public Long vertexJoin(Long vertexValue, DummyCustomParameterizedType<Float> inputValue) {
+			return (long) inputValue.getIntField();
+		}
     }
 }


[04/10] flink git commit: [FLINK-2857] [gelly] Improve Gelly API and documentation. - Improve javadocs of Graph creation methods - Add fromTuple2 creation methods - Rename mapper parameters to vertexInitializer. - Improve javadocs and parameter names of

Posted by fh...@apache.org.
[FLINK-2857] [gelly] Improve Gelly API and documentation.
- Improve javadocs of Graph creation methods
- Add fromTuple2 creation methods
- Rename mapper parameters to vertexInitializer.
- Improve javadocs and parameter names of joinWith* methods
- Improve javadocs of neighborhood methods
- Update docs to reflect api changes

This closes #1263


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

Branch: refs/heads/master
Commit: 640e63beef0b60891178affc7a6e8f0d01a5d000
Parents: da248b1
Author: vasia <va...@apache.org>
Authored: Fri Oct 16 11:11:40 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 19 15:39:37 2015 +0200

----------------------------------------------------------------------
 docs/libs/gelly_guide.md                        |  34 +-
 .../org/apache/flink/graph/scala/Graph.scala    | 372 ++++++++------
 .../operations/GraphCreationWithCsvITCase.scala |   2 +-
 .../test/operations/JoinWithEdgesITCase.scala   |   7 +-
 .../operations/JoinWithVerticesITCase.scala     |   8 +-
 .../apache/flink/graph/EdgeJoinFunction.java    |  45 ++
 .../org/apache/flink/graph/EdgesFunction.java   |  19 +
 .../graph/EdgesFunctionWithVertexValue.java     |  20 +-
 .../main/java/org/apache/flink/graph/Graph.java | 498 +++++++++++--------
 .../apache/flink/graph/NeighborsFunction.java   |  19 +
 .../graph/NeighborsFunctionWithVertexValue.java |  20 +
 .../apache/flink/graph/ReduceEdgesFunction.java |   9 +
 .../flink/graph/ReduceNeighborsFunction.java    |   9 +
 .../apache/flink/graph/VertexJoinFunction.java  |  43 ++
 .../graph/example/EuclideanGraphWeighing.java   |   8 +-
 .../graph/example/JaccardSimilarityMeasure.java |   8 +-
 .../flink/graph/example/MusicProfiles.java      |   7 +-
 .../apache/flink/graph/library/GSAPageRank.java |  11 +-
 .../flink/graph/library/GSATriangleCount.java   |  10 +-
 .../apache/flink/graph/library/PageRank.java    |  11 +-
 .../test/operations/GraphCreationITCase.java    |  59 +++
 .../test/operations/JoinWithEdgesITCase.java    |  41 +-
 .../test/operations/JoinWithVerticesITCase.java |  40 +-
 23 files changed, 887 insertions(+), 413 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/docs/libs/gelly_guide.md
----------------------------------------------------------------------
diff --git a/docs/libs/gelly_guide.md b/docs/libs/gelly_guide.md
index 13d304d..646ec7f 100644
--- a/docs/libs/gelly_guide.md
+++ b/docs/libs/gelly_guide.md
@@ -149,6 +149,30 @@ val graph = Graph.fromDataSet(vertices, edges, env)
 </div>
 </div>
 
+* from a `DataSet` of `Tuple2` representing the edges. Gelly will convert each `Tuple2` to an `Edge`, where the first field will be the source ID and the second field will be the target ID. Both vertex and edge values will be set to `NullValue`.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+DataSet<Tuple2<String, String>> edges = ...
+
+Graph<String, NullValue, NullValue> graph = Graph.fromTuple2DataSet(edges, env);
+{% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val edges: DataSet[(String, String)] = ...
+
+val graph = Graph.fromTuple2DataSet(edges, env)
+{% endhighlight %}
+</div>
+</div>
+
 * from a `DataSet` of `Tuple3` and an optional `DataSet` of `Tuple2`. In this case, Gelly will convert each `Tuple3` to an `Edge`, where the first field will be the source ID, the second field will be the target ID and the third field will be the edge value. Equivalently, each `Tuple2` will be converted to a `Vertex`, where the first field will be the vertex ID and the second field will be the vertex value:
 
 <div class="codetabs" markdown="1">
@@ -424,8 +448,8 @@ graph.subgraph((vertex => vertex.getValue > 0), (edge => edge.getValue < 0))
     <img alt="Filter Transformations" width="80%" src="fig/gelly-filter.png"/>
 </p>
 
-* <strong>Join</strong>: Gelly provides specialized methods for joining the vertex and edge datasets with other input datasets. `joinWithVertices` joins the vertices with a `Tuple2` input data set. The join is performed using the vertex ID and the first field of the `Tuple2` input as the join keys. The method returns a new `Graph` where the vertex values have been updated according to a provided user-defined map function.
-Similarly, an input dataset can be joined with the edges, using one of three methods. `joinWithEdges` expects an input `DataSet` of `Tuple3` and joins on the composite key of both source and target vertex IDs. `joinWithEdgesOnSource` expects a `DataSet` of `Tuple2` and joins on the source key of the edges and the first attribute of the input dataset and `joinWithEdgesOnTarget` expects a `DataSet` of `Tuple2` and joins on the target key of the edges and the first attribute of the input dataset. All three methods apply a map function on the edge and the input data set values.
+* <strong>Join</strong>: Gelly provides specialized methods for joining the vertex and edge datasets with other input datasets. `joinWithVertices` joins the vertices with a `Tuple2` input data set. The join is performed using the vertex ID and the first field of the `Tuple2` input as the join keys. The method returns a new `Graph` where the vertex values have been updated according to a provided user-defined transformation function.
+Similarly, an input dataset can be joined with the edges, using one of three methods. `joinWithEdges` expects an input `DataSet` of `Tuple3` and joins on the composite key of both source and target vertex IDs. `joinWithEdgesOnSource` expects a `DataSet` of `Tuple2` and joins on the source key of the edges and the first attribute of the input dataset and `joinWithEdgesOnTarget` expects a `DataSet` of `Tuple2` and joins on the target key of the edges and the first attribute of the input dataset. All three methods apply a transformation function on the edge and the input data set values.
 Note that if the input dataset contains a key multiple times, all Gelly join methods will only consider the first value encountered.
 
 <div class="codetabs" markdown="1">
@@ -437,9 +461,9 @@ DataSet<Tuple2<Long, Long>> vertexOutDegrees = network.outDegrees();
 
 // assign the transition probabilities as the edge weights
 Graph<Long, Double, Double> networkWithWeights = network.joinWithEdgesOnSource(vertexOutDegrees,
-				new MapFunction<Tuple2<Double, Long>, Double>() {
-					public Double map(Tuple2<Double, Long> value) {
-						return value.f0 / value.f1;
+				new VertexJoinFunction<Double, Long>() {
+					public Double vertexJoin(Double vertexValue, Long inputValue) {
+						return vertexValue / inputValue;
 					}
 				});
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
index 28f3f12..e51453e 100644
--- a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
@@ -54,12 +54,13 @@ object Graph {
   /**
   * Creates a graph from a DataSet of edges.
   * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
+  * vertexValueInitializer map function to the vertex ids.
   */
   def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: DataSet[Edge[K, EV]], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](edges.javaSet, mapper, env.getJavaEnv))
+  TypeInformation : ClassTag](edges: DataSet[Edge[K, EV]],
+  vertexValueInitializer: MapFunction[K, VV], env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](edges.javaSet, vertexValueInitializer,
+      env.getJavaEnv))
   }
 
   /**
@@ -84,16 +85,22 @@ object Graph {
   /**
   * Creates a graph from a Seq of edges.
   * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
+  * vertexValueInitializer map function to the vertex ids.
   */
   def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: Seq[Edge[K, EV]], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromCollection[K, VV, EV](edges.asJavaCollection, mapper, env.getJavaEnv))
+  TypeInformation : ClassTag](edges: Seq[Edge[K, EV]], vertexValueInitializer: MapFunction[K, VV],
+  env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromCollection[K, VV, EV](edges.asJavaCollection, vertexValueInitializer,
+      env.getJavaEnv))
   }
 
   /**
-  * Creates a Graph from a DataSets of Tuples.
+   * Creates a graph from DataSets of tuples for vertices and for edges.
+   * The first field of the Tuple2 vertex object will become the vertex ID
+   * and the second field will become the vertex value.
+   * The first field of the Tuple3 object for edges will become the source ID,
+   * the second field will become the target ID, and the third field will become
+   * the edge value. 
   */
   def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
   TypeInformation : ClassTag](vertices: DataSet[(K, VV)], edges: DataSet[(K, K, EV)],
@@ -101,11 +108,14 @@ object Graph {
     val javaTupleVertices = vertices.map(v => new jtuple.Tuple2(v._1, v._2)).javaSet
     val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
     wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleVertices, javaTupleEdges,
-        env.getJavaEnv))
+      env.getJavaEnv))
   }
 
   /**
   * Creates a Graph from a DataSet of Tuples representing the edges.
+  * The first field of the Tuple3 object for edges will become the source ID,
+  * the second field will become the target ID, and the third field will become
+  * the edge value. 
   * Vertices are created automatically and their values are set to NullValue.
   */
   def fromTupleDataSet[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
@@ -116,14 +126,45 @@ object Graph {
 
   /**
   * Creates a Graph from a DataSet of Tuples representing the edges.
+  * The first field of the Tuple3 object for edges will become the source ID,
+  * the second field will become the target ID, and the third field will become
+  * the edge value. 
   * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
+  * vertexValueInitializer map function to the vertex ids.
   */
   def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: DataSet[(K, K, EV)], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
+  TypeInformation : ClassTag](edges: DataSet[(K, K, EV)],
+  vertexValueInitializer: MapFunction[K, VV], env: ExecutionEnvironment): Graph[K, VV, EV] = {
     val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
-    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleEdges, mapper, env.getJavaEnv))
+    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleEdges, vertexValueInitializer,
+      env.getJavaEnv))
+  }
+
+    /**
+  * Creates a Graph from a DataSet of Tuple2's representing the edges.
+  * The first field of the Tuple2 object for edges will become the source ID,
+  * the second field will become the target ID. The edge value will be set to NullValue.
+  * Vertices are created automatically and their values are set to NullValue.
+  */
+  def fromTuple2DataSet[K: TypeInformation : ClassTag](edges: DataSet[(K, K)],
+  env: ExecutionEnvironment): Graph[K, NullValue, NullValue] = {
+    val javaTupleEdges = edges.map(v => new jtuple.Tuple2(v._1, v._2)).javaSet
+    wrapGraph(jg.Graph.fromTuple2DataSet[K](javaTupleEdges, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a DataSet of Tuple2's representing the edges.
+  * The first field of the Tuple2 object for edges will become the source ID,
+  * the second field will become the target ID. The edge value will be set to NullValue.
+  * Vertices are created automatically and their values are set by applying the provided
+  * vertexValueInitializer map function to the vertex IDs.
+  */
+  def fromTuple2DataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag]
+  (edges: DataSet[(K, K)], vertexValueInitializer: MapFunction[K, VV],
+  env: ExecutionEnvironment): Graph[K, VV, NullValue] = {
+    val javaTupleEdges = edges.map(v => new jtuple.Tuple2(v._1, v._2)).javaSet
+    wrapGraph(jg.Graph.fromTuple2DataSet[K, VV](javaTupleEdges, vertexValueInitializer,
+      env.getJavaEnv))
   }
 
   /**
@@ -160,7 +201,8 @@ object Graph {
   * edges file.
   * @param includedFieldsEdges The fields in the edges file that should be read.
   * By default all fields are read.
-  * @param mapper If no vertex values are provided, this mapper can be used to initialize them.
+  * @param vertexValueInitializer If no vertex values are provided,
+  * this mapper can be used to initialize them, by applying a map transformation on the vertex IDs.
   * 
   */
   // scalastyle:off
@@ -186,7 +228,7 @@ object Graph {
       ignoreCommentsEdges: String = null,
       lenientEdges: Boolean = false,
       includedFieldsEdges: Array[Int] = null,
-      mapper: MapFunction[K, VV] = null) = {
+      vertexValueInitializer: MapFunction[K, VV] = null) = {
 
     // with vertex and edge values
     if (readVertices && hasEdgeValues) {
@@ -229,8 +271,8 @@ object Graph {
         includedFieldsEdges)
 
       // initializer provided
-      if (mapper != null) {
-        fromTupleDataSet[K, VV, EV](edges, mapper, env)
+      if (vertexValueInitializer != null) {
+        fromTupleDataSet[K, VV, EV](edges, vertexValueInitializer, env)
       }
       else {
         fromTupleDataSet[K, EV](edges, env) 
@@ -243,8 +285,8 @@ object Graph {
       lenientEdges, includedFieldsEdges).map(edge => (edge._1, edge._2, NullValue.getInstance))
 
       // no initializer provided
-      if (mapper != null) {
-        fromTupleDataSet[K, VV, NullValue](edges, mapper, env)
+      if (vertexValueInitializer != null) {
+        fromTupleDataSet[K, VV, NullValue](edges, vertexValueInitializer, env)
       }
       else {
         fromTupleDataSet[K, NullValue](edges, env) 
@@ -369,185 +411,215 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) {
   }
 
   /**
-   * Joins the vertex DataSet of this graph with an input DataSet and applies
-   * a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @return a new graph where the vertex values have been updated.
-   */
-  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: MapFunction[
-    (VV, T), VV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
-      override def map(value: jtuple.Tuple2[VV, T]): VV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
+   * Joins the vertex DataSet of this graph with an input Tuple2 DataSet and applies
+   * a user-defined transformation on the values of the matched records.
+   * The vertex ID and the first field of the Tuple2 DataSet are used as the join keys.
+   * 
+   * @param inputDataSet the Tuple2 DataSet to join with.
+   * The first field of the Tuple2 is used as the join key and the second field is passed
+   * as a parameter to the transformation function.
+   * @param vertexJoinFunction the transformation function to apply.
+   * The first parameter is the current vertex value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @return a new Graph, where the vertex values have been updated according to the
+   * result of the vertexJoinFunction.
+   * 
+   * @tparam T the type of the second field of the input Tuple2 DataSet.
+  */
+  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)],
+  vertexJoinFunction: VertexJoinFunction[VV, T]): Graph[K, VV, EV] = {
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, vertexJoinFunction))
   }
 
   /**
-   * Joins the vertex DataSet of this graph with an input DataSet and applies
-   * a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @return a new graph where the vertex values have been updated.
-   */
+   * Joins the vertex DataSet of this graph with an input Tuple2 DataSet and applies
+   * a user-defined transformation on the values of the matched records.
+   * The vertex ID and the first field of the Tuple2 DataSet are used as the join keys.
+   * 
+   * @param inputDataSet the Tuple2 DataSet to join with.
+   * The first field of the Tuple2 is used as the join key and the second field is passed
+   * as a parameter to the transformation function.
+   * @param fun the transformation function to apply.
+   * The first parameter is the current vertex value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @return a new Graph, where the vertex values have been updated according to the
+   * result of the vertexJoinFunction.
+   * 
+   * @tparam T the type of the second field of the input Tuple2 DataSet.
+  */
   def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (VV, T) => VV):
   Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
+    val newVertexJoin = new VertexJoinFunction[VV, T]() {
       val cleanFun = clean(fun)
 
-      override def map(value: jtuple.Tuple2[VV, T]): VV = {
-        cleanFun(value.f0, value.f1)
+      override def vertexJoin(vertexValue: VV, inputValue: T): VV = {
+        cleanFun(vertexValue, inputValue)
       }
     }
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newVertexJoin))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on a composite key of both
-   * source and target and applies a UDF on the resulted values.
-   *
+   * Joins the edge DataSet with an input DataSet on the composite key of both
+   * source and target IDs and applies a user-defined transformation on the values
+   * of the matched records. The first two fields of the input DataSet are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], mapper: MapFunction[
-    (EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
+   * The first two fields of the Tuple3 are used as the composite join key
+   * and the third field is passed as a parameter to the transformation function.
+   * @param edgeJoinFunction the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple3 from the input DataSet.
+   * 
+   * @tparam T the type of the third field of the input Tuple3 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
+  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)],
+  edgeJoinFunction: EdgeJoinFunction[EV, T]): Graph[K, VV, EV] = {
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
       scalatuple._2, scalatuple._3)).javaSet
-    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, edgeJoinFunction))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on a composite key of both
-   * source and target and applies a UDF on the resulted values.
-   *
+   * Joins the edge DataSet with an input DataSet on the composite key of both
+   * source and target IDs and applies a user-defined transformation on the values
+   * of the matched records. The first two fields of the input DataSet are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
+   * The first two fields of the Tuple3 are used as the composite join key
+   * and the third field is passed as a parameter to the transformation function.
+   * @param fun the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple3 from the input DataSet.
+   * 
+   * @tparam T the type of the third field of the input Tuple3 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
   def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], fun: (EV, T) => EV):
   Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+    val newEdgeJoin = new EdgeJoinFunction[EV, T]() {
       val cleanFun = clean(fun)
 
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
+      override def edgeJoin(edgeValue: EV, inputValue: T): EV = {
+        cleanFun(edgeValue, inputValue)
       }
     }
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
       scalatuple._2, scalatuple._3)).javaSet
-    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newEdgeJoin))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on the source key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. In case the inputDataSet contains the same key more
-   * than once, only the first value will be considered.
-   *
+   * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+   * on the values of the matched records.
+   * The source ID of the edges input and the first field of the input DataSet
+   * are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
-  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
+   * The first field of the Tuple2 is used as the join key
+   * and the second field is passed as a parameter to the transformation function.
+   * @param edgeJoinFunction the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @tparam T the type of the second field of the input Tuple2 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
+  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)],
+  edgeJoinFunction: EdgeJoinFunction[EV, T]): Graph[K, VV, EV] = {
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, edgeJoinFunction))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on the source key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. In case the inputDataSet contains the same key more
-   * than once, only the first value will be considered.
-   *
+   * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+   * on the values of the matched records.
+   * The source ID of the edges input and the first field of the input DataSet
+   * are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
+   * The first field of the Tuple2 is used as the join key
+   * and the second field is passed as a parameter to the transformation function.
+   * @param fun the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @tparam T the type of the second field of the input Tuple2 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
   def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
     EV): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+    val newEdgeJoin = new EdgeJoinFunction[EV, T]() {
       val cleanFun = clean(fun)
 
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
+      override def edgeJoin(edgeValue: EV, inputValue: T): EV = {
+        cleanFun(edgeValue, inputValue)
       }
     }
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newEdgeJoin))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on the target key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. Should the inputDataSet contain the same key more
-   * than once, only the first value will be considered.
-   *
+   * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+   * on the values of the matched records.
+   * The target ID of the edges input and the first field of the input DataSet
+   * are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
-  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
+   * The first field of the Tuple2 is used as the join key
+   * and the second field is passed as a parameter to the transformation function.
+   * @param edgeJoinFunction the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @param T the type of the second field of the input Tuple2 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
+  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)],
+  edgeJoinFunction: EdgeJoinFunction[EV, T]): Graph[K, VV, EV] = {
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, edgeJoinFunction))
   }
 
   /**
-   * Joins the edge DataSet with an input DataSet on the target key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. Should the inputDataSet contain the same key more
-   * than once, only the first value will be considered.
-   *
+   * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+   * on the values of the matched records.
+   * The target ID of the edges input and the first field of the input DataSet
+   * are used as join keys.
+   * 
    * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
+   * The first field of the Tuple2 is used as the join key
+   * and the second field is passed as a parameter to the transformation function.
+   * @param fun the transformation function to apply.
+   * The first parameter is the current edge value and the second parameter is the value
+   * of the matched Tuple2 from the input DataSet.
+   * @param T the type of the second field of the input Tuple2 DataSet.
+   * @return a new Graph, where the edge values have been updated according to the
+   * result of the edgeJoinFunction.
+  */
   def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
     EV): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+    val newEdgeJoin = new EdgeJoinFunction[EV, T]() {
       val cleanFun = clean(fun)
 
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
+      override def edgeJoin(edgeValue: EV, inputValue:T): EV = {
+        cleanFun(edgeValue, inputValue)
       }
     }
     val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
       scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
+    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newEdgeJoin))
   }
 
   /**
@@ -896,12 +968,17 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) {
   }
 
   /**
-   * Compute an aggregate over the neighbor values of each
-   * vertex.
-   *
-   * @param reduceNeighborsFunction the function to apply to the neighborhood
-   * @param direction               the edge direction (in-, out-, all-)
-   * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
+   * Compute a reduce transformation over the neighbors' vertex values of each vertex.
+   * For each vertex, the transformation consecutively calls a
+   * {@link ReduceNeighborsFunction} until only a single value for each vertex remains.
+   * The {@link ReduceNeighborsFunction} combines a pair of neighbor vertex values
+   * into one new value of the same type.
+   * 
+   * @param reduceNeighborsFunction the reduce function to apply to the neighbors of each vertex.
+   * @param direction the edge direction (in-, out-, all-)
+   * @return a Dataset of Tuple2, with one tuple per vertex.
+   * The first field of the Tuple2 is the vertex ID and the second field
+   * is the aggregate value computed by the provided {@link ReduceNeighborsFunction}.
    */
   def reduceOnNeighbors(reduceNeighborsFunction: ReduceNeighborsFunction[VV], direction:
   EdgeDirection): DataSet[(K, VV)] = {
@@ -910,13 +987,18 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) {
   }
 
   /**
-   * Compute an aggregate over the edge values of each vertex.
-   *
-   * @param reduceEdgesFunction the function to apply to the neighborhood
-   * @param direction           the edge direction (in-, out-, all-)
-   * @return a Dataset containing one value per vertex(vertex key, aggegate edge value)
-   * @throws IllegalArgumentException
-   */
+   * Compute a reduce transformation over the neighbors' vertex values of each vertex.
+   * For each vertex, the transformation consecutively calls a
+   * {@link ReduceNeighborsFunction} until only a single value for each vertex remains.
+   * The {@link ReduceNeighborsFunction} combines a pair of neighbor vertex values
+   * into one new value of the same type.
+   * 
+   * @param reduceNeighborsFunction the reduce function to apply to the neighbors of each vertex.
+   * @param direction the edge direction (in-, out-, all-)
+   * @return a Dataset of Tuple2, with one tuple per vertex.
+   * The first field of the Tuple2 is the vertex ID and the second field
+   * is the aggregate value computed by the provided {@link ReduceNeighborsFunction}.
+  */
   def reduceOnEdges(reduceEdgesFunction: ReduceEdgesFunction[EV], direction: EdgeDirection):
   DataSet[(K, EV)] = {
     wrap(jgraph.reduceOnEdges(reduceEdgesFunction, direction)).map(jtuple => (jtuple.f0,

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
index 6ceaf16..a963845 100644
--- a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
@@ -101,7 +101,7 @@ MultipleProgramsTestBase(mode) {
     val graph = Graph.fromCsvReader[Long, Double, Long](
         readVertices = false,
         pathEdges = edgesSplit.getPath.toString,
-        mapper = new VertexDoubleIdAssigner(),
+        vertexValueInitializer = new VertexDoubleIdAssigner(),
         env = env)
     
     val result = graph.getTriplets.collect()

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
index 3dc90fc..83fa61b 100644
--- a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
@@ -30,6 +30,7 @@ import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.{After, Before, Rule, Test}
 import _root_.scala.collection.JavaConverters._
+import org.apache.flink.graph.EdgeJoinFunction
 
 @RunWith(classOf[Parameterized])
 class JoinWithEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
@@ -127,10 +128,10 @@ MultipleProgramsTestBase(mode) {
   }
 
 
-  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
+  final class AddValuesMapper extends EdgeJoinFunction[Long, Long] {
     @throws(classOf[Exception])
-    def map(tuple: (Long, Long)): Long = {
-      tuple._1 + tuple._2
+    def edgeJoin(edgeValue: Long, inputValue: Long): Long = {
+      edgeValue + inputValue
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
index 98ee8b6..f2beb7b 100644
--- a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
@@ -29,6 +29,7 @@ import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.{After, Before, Rule, Test}
 import _root_.scala.collection.JavaConverters._
+import org.apache.flink.graph.VertexJoinFunction
 
 @RunWith(classOf[Parameterized])
 class JoinWithVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
@@ -63,11 +64,10 @@ MultipleProgramsTestBase(mode) {
     TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
   }
 
-
-  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
+  final class AddValuesMapper extends VertexJoinFunction[Long, Long] {
     @throws(classOf[Exception])
-    def map(tuple: (Long, Long)): Long = {
-      tuple._1 + tuple._2
+    def vertexJoin(vertexValue: Long, inputValue: Long): Long = {
+      vertexValue + inputValue
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeJoinFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeJoinFunction.java
new file mode 100644
index 0000000..68d6e53
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeJoinFunction.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.graph;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+
+/**
+ * Interface to be implemented by the transformation function
+ * applied in {@link Graph#joinWithEdges(DataSet, EdgeJoinFunction)},
+ * {@link Graph#joinWithEdgesOnSource(DataSet, EdgeJoinFunction)}, and
+ * {@link Graph#joinWithEdgesOnTarget(DataSet, EdgeJoinFunction)} methods.
+ *
+ * @param <EV> the edge value type
+ * @param <T> the input value type
+ */
+public interface EdgeJoinFunction<EV, T> extends Function, Serializable {
+
+	/**
+	 * Applies a transformation on the current edge value
+	 * and the value of the matched tuple of the input DataSet.
+	 * 
+	 * @param edgeValue the current edge value
+	 * @param inputValue the value of the matched Tuple2 input
+	 * @return the new edge value
+	 */
+	EV edgeJoin(EV edgeValue, T inputValue) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
index bf1d6a2..07e14e9 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
@@ -34,5 +34,24 @@ import org.apache.flink.util.Collector;
  */
 public interface EdgesFunction<K, EV, O> extends Function, Serializable {
 
+	/**
+	 * This method is called per vertex and can iterate over all of its neighboring edges
+	 * with the specified direction.
+	 * <p>
+	 * If called with {@link EdgeDirection#OUT} the group will contain
+	 * the out-edges of the grouping vertex.
+	 * If called with {@link EdgeDirection#IN} the group will contain
+	 * the in-edges of the grouping vertex.
+	 * If called with {@link EdgeDirection#ALL} the group will contain
+	 * all edges of the grouping vertex.
+	 * <p>
+	 * The method can emit any number of output elements, including none.
+	 * 
+	 * @param edges the neighboring edges of the grouping vertex.
+	 * The first filed of each Tuple2 is the ID of the grouping vertex.
+	 * The second field is the neighboring edge.
+	 * @param out the collector to emit results to
+	 * @throws Exception
+	*/
 	void iterateEdges(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<O> out) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
index 0b0ab0e..645bd7c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
@@ -35,5 +35,23 @@ import org.apache.flink.util.Collector;
  */
 public interface EdgesFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
 
-	void iterateEdges(Vertex<K, VV> v, Iterable<Edge<K, EV>> edges, Collector<O> out) throws Exception;
+	/**
+	 * This method is called per vertex and can iterate over all of its neighboring edges
+	 * with the specified direction.
+	 * <p>
+	 * If called with {@link EdgeDirection#OUT} the group will contain
+	 * the out-edges of the grouping vertex.
+	 * If called with {@link EdgeDirection#IN} the group will contain
+	 * the in-edges of the grouping vertex.
+	 * If called with {@link EdgeDirection#ALL} the group will contain
+	 * all edges of the grouping vertex.
+	 * <p>
+	 * The method can emit any number of output elements, including none.
+	 * 
+	 * @param vertex the grouping vertex
+	 * @param edges the neighboring edges of the grouping vertex.
+	 * @param out the collector to emit results to
+	 * @throws Exception
+	*/
+	void iterateEdges(Vertex<K, VV> vertex, Iterable<Edge<K, EV>> edges, Collector<O> out) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
index b24f749..6015be4 100755
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
@@ -111,11 +111,11 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a Collection of edges, vertices are induced from the
-	 * edges. Vertices are created automatically and their values are set to
+	 * Creates a graph from a Collection of edges.
+	 * Vertices are created automatically and their values are set to
 	 * NullValue.
 	 * 
-	 * @param edges a Collection of vertices.
+	 * @param edges a Collection of edges.
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
@@ -126,20 +126,20 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a Collection of edges, vertices are induced from the
-	 * edges and vertex values are calculated by a mapper function. Vertices are
-	 * created automatically and their values are set by applying the provided
-	 * map function to the vertex ids.
+	 * Creates a graph from a Collection of edges.
+	 * Vertices are created automatically and their values are set 
+	 * by applying the provided map function to the vertex IDs.
 	 * 
 	 * @param edges a Collection of edges.
-	 * @param mapper the mapper function.
+	 * @param vertexValueInitializer a map function that initializes the vertex values.
+	 * It allows to apply a map transformation on the vertex ID to produce an initial vertex value. 
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
 	public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Edge<K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
 
-		return fromDataSet(context.fromCollection(edges), mapper, context);
+		return fromDataSet(context.fromCollection(edges), vertexValueInitializer, context);
 	}
 
 	/**
@@ -157,8 +157,8 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a DataSet of edges, vertices are induced from the
-	 * edges. Vertices are created automatically and their values are set to
+	 * Creates a graph from a DataSet of edges.
+	 * Vertices are created automatically and their values are set to
 	 * NullValue.
 	 * 
 	 * @param edges a DataSet of edges.
@@ -183,23 +183,23 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a DataSet of edges, vertices are induced from the
-	 * edges and vertex values are calculated by a mapper function. Vertices are
-	 * created automatically and their values are set by applying the provided
-	 * map function to the vertex ids.
+	 * Creates a graph from a DataSet of edges.
+	 * Vertices are created automatically and their values are set
+	 * by applying the provided map function to the vertex IDs.
 	 * 
 	 * @param edges a DataSet of edges.
-	 * @param mapper the mapper function.
+	 * @param vertexValueInitializer the mapper function that initializes the vertex values.
+	 * It allows to apply a map transformation on the vertex ID to produce an initial vertex value.
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
 	public static <K, VV, EV> Graph<K, VV, EV> fromDataSet(DataSet<Edge<K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
 
 		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
 
 		TypeInformation<VV> valueType = TypeExtractor.createTypeInfo(
-				MapFunction.class, mapper.getClass(), 1, null, null);
+				MapFunction.class, vertexValueInitializer.getClass(), 1, null, null);
 
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		TypeInformation<Vertex<K, VV>> returnType = (TypeInformation<Vertex<K, VV>>) new TupleTypeInfo(
@@ -209,7 +209,7 @@ public class Graph<K, VV, EV> {
 				.flatMap(new EmitSrcAndTargetAsTuple1<K, EV>()).distinct()
 				.map(new MapFunction<Tuple1<K>, Vertex<K, VV>>() {
 					public Vertex<K, VV> map(Tuple1<K> value) throws Exception {
-						return new Vertex<K, VV>(value.f0, mapper.map(value.f0));
+						return new Vertex<K, VV>(value.f0, vertexValueInitializer.map(value.f0));
 					}
 				}).returns(returnType).withForwardedFields("f0");
 
@@ -226,13 +226,17 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a DataSet of Tuple objects for vertices and edges.
-	 * 
-	 * Vertices with value are created from Tuple2, Edges with value are created
-	 * from Tuple3.
+	 * Creates a graph from a DataSet of Tuple2 objects for vertices and 
+	 * Tuple3 objects for edges.
+	 * <p>
+	 * The first field of the Tuple2 vertex object will become the vertex ID
+	 * and the second field will become the vertex value.
+	 * The first field of the Tuple3 object for edges will become the source ID,
+	 * the second field will become the target ID, and the third field will become
+	 * the edge value.
 	 * 
-	 * @param vertices a DataSet of Tuple2.
-	 * @param edges a DataSet of Tuple3.
+	 * @param vertices a DataSet of Tuple2 representing the vertices.
+	 * @param edges a DataSet of Tuple3 representing the edges.
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
@@ -245,13 +249,15 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
-	 * induced from the edges.
+	 * Creates a graph from a DataSet of Tuple3 objects for edges.
+	 * <p>
+	 * The first field of the Tuple3 object will become the source ID,
+	 * the second field will become the target ID, and the third field will become
+	 * the edge value.
+	 * <p>
+	 * Vertices are created automatically and their values are set to NullValue.
 	 * 
-	 * Edges with value are created from Tuple3. Vertices are created
-	 * automatically and their values are set to NullValue.
-	 * 
-	 * @param edges a DataSet of Tuple3.
+	 * @param edges a DataSet of Tuple3 representing the edges.
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
@@ -263,22 +269,78 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
-	 * induced from the edges and vertex values are calculated by a mapper
-	 * function. Edges with value are created from Tuple3. Vertices are created
-	 * automatically and their values are set by applying the provided map
-	 * function to the vertex ids.
+	 * Creates a graph from a DataSet of Tuple3 objects for edges.
+	 * <p>
+	 * Each Tuple3 will become one Edge, where the source ID will be the first field of the Tuple2,
+	 * the target ID will be the second field of the Tuple2
+	 * and the Edge value will be the third field of the Tuple3.
+	 * <p>
+	 * Vertices are created automatically and their values are initialized
+	 * by applying the provided vertexValueInitializer map function to the vertex IDs.
 	 * 
 	 * @param edges a DataSet of Tuple3.
-	 * @param mapper the mapper function.
+	 * @param vertexValueInitializer the mapper function that initializes the vertex values.
+	 * It allows to apply a map transformation on the vertex ID to produce an initial vertex value.
 	 * @param context the flink execution environment.
 	 * @return the newly created graph.
 	 */
 	public static <K, VV, EV> Graph<K, VV, EV> fromTupleDataSet(DataSet<Tuple3<K, K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
 
 		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
-		return fromDataSet(edgeDataSet, mapper, context);
+		return fromDataSet(edgeDataSet, vertexValueInitializer, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of Tuple2 objects for edges.
+	 * Each Tuple2 will become one Edge, where the source ID will be the first field of the Tuple2
+	 * and the target ID will be the second field of the Tuple2.
+	 * <p>
+	 * Edge value types and Vertex values types will be set to NullValue.
+	 * 
+	 * @param edges a DataSet of Tuple2.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K> Graph<K, NullValue, NullValue> fromTuple2DataSet(DataSet<Tuple2<K, K>> edges,
+			ExecutionEnvironment context) {
+
+		DataSet<Edge<K, NullValue>> edgeDataSet = edges.map(
+				new MapFunction<Tuple2<K, K>, Edge<K, NullValue>>() {
+
+					public Edge<K, NullValue> map(Tuple2<K, K> input) {
+						return new Edge<K, NullValue>(input.f0, input.f1, NullValue.getInstance());
+					}
+		}).withForwardedFields("f0; f1");
+		return fromDataSet(edgeDataSet, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of Tuple2 objects for edges.
+	 * Each Tuple2 will become one Edge, where the source ID will be the first field of the Tuple2
+	 * and the target ID will be the second field of the Tuple2.
+	 * <p>
+	 * Edge value types will be set to NullValue.
+	 * Vertex values can be initialized by applying a user-defined map function on the vertex IDs.
+	 * 
+	 * @param edges a DataSet of Tuple2, where the first field corresponds to the source ID
+	 * and the second field corresponds to the target ID.
+	 * @param vertexValueInitializer the mapper function that initializes the vertex values.
+	 * It allows to apply a map transformation on the vertex ID to produce an initial vertex value.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV> Graph<K, VV, NullValue> fromTuple2DataSet(DataSet<Tuple2<K, K>> edges,
+			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
+
+		DataSet<Edge<K, NullValue>> edgeDataSet = edges.map(
+				new MapFunction<Tuple2<K, K>, Edge<K, NullValue>>() {
+
+					public Edge<K, NullValue> map(Tuple2<K, K> input) {
+						return new Edge<K, NullValue>(input.f0, input.f1, NullValue.getInstance());
+					}
+				}).withForwardedFields("f0; f1");
+		return fromDataSet(edgeDataSet, vertexValueInitializer, context);
 	}
 
 	/**
@@ -318,10 +380,11 @@ public class Graph<K, VV, EV> {
 
 	/** 
 	 * Creates a graph from a CSV file of edges. Vertices will be created automatically and
-	 * Vertex values are set by the provided mapper.
+	 * Vertex values can be initialized using a user-defined mapper.
 	 *
 	 * @param edgesPath a path to a CSV file with the Edge data
-	 * @param mapper the mapper function.
+	 * @param vertexValueInitializer the mapper function that initializes the vertex values.
+	 * It allows to apply a map transformation on the vertex ID to produce an initial vertex value.
 	 * @param context the execution environment.
 	 * @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
 	 * on which calling methods to specify types of the Vertex ID, Vertex Value and Edge value returns a Graph.
@@ -332,8 +395,8 @@ public class Graph<K, VV, EV> {
 	 * {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
 	 */
 	public static <K, VV> GraphCsvReader fromCsvReader(String edgesPath,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-		return new GraphCsvReader(edgesPath, mapper, context);
+			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
+		return new GraphCsvReader(edgesPath, vertexValueInitializer, context);
 	}
 
 	/**
@@ -501,29 +564,37 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Joins the vertex DataSet of this graph with an input DataSet and applies
-	 * a UDF on the resulted values.
+	 * Joins the vertex DataSet of this graph with an input Tuple2 DataSet and applies
+	 * a user-defined transformation on the values of the matched records.
+	 * The vertex ID and the first field of the Tuple2 DataSet are used as the join keys.
 	 * 
-	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @return a new graph where the vertex values have been updated.
-	 */
+	 * @param inputDataSet the Tuple2 DataSet to join with.
+	 * The first field of the Tuple2 is used as the join key and the second field is passed
+	 * as a parameter to the transformation function. 
+	 * @param vertexJoinFunction the transformation function to apply.
+	 * The first parameter is the current vertex value and the second parameter is the value
+	 * of the matched Tuple2 from the input DataSet.
+	 * @return a new Graph, where the vertex values have been updated according to the
+	 * result of the vertexJoinFunction.
+	 * 
+	 * @param <T> the type of the second field of the input Tuple2 DataSet.
+	*/
 	public <T> Graph<K, VV, EV> joinWithVertices(DataSet<Tuple2<K, T>> inputDataSet, 
-			final MapFunction<Tuple2<VV, T>, VV> mapper) {
+			final VertexJoinFunction<VV, T> vertexJoinFunction) {
 
 		DataSet<Vertex<K, VV>> resultedVertices = this.getVertices()
 				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToVertexValues<K, VV, T>(mapper));
+				.with(new ApplyCoGroupToVertexValues<K, VV, T>(vertexJoinFunction));
 		return new Graph<K, VV, EV>(resultedVertices, this.edges, this.context);
 	}
 
 	private static final class ApplyCoGroupToVertexValues<K, VV, T>
 			implements CoGroupFunction<Vertex<K, VV>, Tuple2<K, T>, Vertex<K, VV>> {
 
-		private MapFunction<Tuple2<VV, T>, VV> mapper;
+		private VertexJoinFunction<VV, T> vertexJoinFunction;
 
-		public ApplyCoGroupToVertexValues(MapFunction<Tuple2<VV, T>, VV> mapper) {
-			this.mapper = mapper;
+		public ApplyCoGroupToVertexValues(VertexJoinFunction<VV, T> mapper) {
+			this.vertexJoinFunction = mapper;
 		}
 
 		@Override
@@ -537,42 +608,46 @@ public class Graph<K, VV, EV> {
 				if (inputIterator.hasNext()) {
 					final Tuple2<K, T> inputNext = inputIterator.next();
 
-					collector.collect(new Vertex<K, VV>(inputNext.f0, mapper
-							.map(new Tuple2<VV, T>(vertexIterator.next().f1,
-									inputNext.f1))));
+					collector.collect(new Vertex<K, VV>(inputNext.f0, vertexJoinFunction
+							.vertexJoin(vertexIterator.next().f1, inputNext.f1)));
 				} else {
 					collector.collect(vertexIterator.next());
 				}
-
 			}
 		}
 	}
 
 	/**
-	 * Joins the edge DataSet with an input DataSet on a composite key of both
-	 * source and target and applies a UDF on the resulted values.
+	 * Joins the edge DataSet with an input DataSet on the composite key of both
+	 * source and target IDs and applies a user-defined transformation on the values
+	 * of the matched records. The first two fields of the input DataSet are used as join keys.
 	 * 
 	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
+	 * The first two fields of the Tuple3 are used as the composite join key
+	 * and the third field is passed as a parameter to the transformation function. 
+	 * @param edgeJoinFunction the transformation function to apply.
+	 * The first parameter is the current edge value and the second parameter is the value
+	 * of the matched Tuple3 from the input DataSet.
+	 * @param <T> the type of the third field of the input Tuple3 DataSet.
+	 * @return a new Graph, where the edge values have been updated according to the
+	 * result of the edgeJoinFunction.
+	*/
 	public <T> Graph<K, VV, EV> joinWithEdges(DataSet<Tuple3<K, K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+			final EdgeJoinFunction<EV, T> edgeJoinFunction) {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(0, 1).equalTo(0, 1)
-				.with(new ApplyCoGroupToEdgeValues<K, EV, T>(mapper));
+				.with(new ApplyCoGroupToEdgeValues<K, EV, T>(edgeJoinFunction));
 		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
 	}
 
 	private static final class ApplyCoGroupToEdgeValues<K, EV, T>
 			implements CoGroupFunction<Edge<K, EV>, Tuple3<K, K, T>, Edge<K, EV>> {
 
-		private MapFunction<Tuple2<EV, T>, EV> mapper;
+		private EdgeJoinFunction<EV, T> edgeJoinFunction;
 
-		public ApplyCoGroupToEdgeValues(MapFunction<Tuple2<EV, T>, EV> mapper) {
-			this.mapper = mapper;
+		public ApplyCoGroupToEdgeValues(EdgeJoinFunction<EV, T> mapper) {
+			this.edgeJoinFunction = mapper;
 		}
 
 		@Override
@@ -587,8 +662,8 @@ public class Graph<K, VV, EV> {
 					final Tuple3<K, K, T> inputNext = inputIterator.next();
 
 					collector.collect(new Edge<K, EV>(inputNext.f0,
-							inputNext.f1, mapper.map(new Tuple2<EV, T>(
-									edgesIterator.next().f2, inputNext.f2))));
+							inputNext.f1, edgeJoinFunction.edgeJoin(
+									edgesIterator.next().f2, inputNext.f2)));
 				} else {
 					collector.collect(edgesIterator.next());
 				}
@@ -597,22 +672,26 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Joins the edge DataSet with an input DataSet on the source key of the
-	 * edges and the first attribute of the input DataSet and applies a UDF on
-	 * the resulted values. In case the inputDataSet contains the same key more
-	 * than once, only the first value will be considered.
+	 * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+	 * on the values of the matched records.
+	 * The source ID of the edges input and the first field of the input DataSet are used as join keys.
 	 * 
 	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
+	 * The first field of the Tuple2 is used as the join key
+	 * and the second field is passed as a parameter to the transformation function. 
+	 * @param edgeJoinFunction the transformation function to apply.
+	 * The first parameter is the current edge value and the second parameter is the value
+	 * of the matched Tuple2 from the input DataSet.
+	 * @param <T> the type of the second field of the input Tuple2 DataSet.
+	 * @return a new Graph, where the edge values have been updated according to the
+	 * result of the edgeJoinFunction.
+	*/
 	public <T> Graph<K, VV, EV> joinWithEdgesOnSource(DataSet<Tuple2<K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+			final EdgeJoinFunction<EV, T> edgeJoinFunction) {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(edgeJoinFunction));
 
 		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
 	}
@@ -620,11 +699,10 @@ public class Graph<K, VV, EV> {
 	private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>
 			implements CoGroupFunction<Edge<K, EV>, Tuple2<K, T>, Edge<K, EV>> {
 
-		private MapFunction<Tuple2<EV, T>, EV> mapper;
+		private EdgeJoinFunction<EV, T> edgeJoinFunction;
 
-		public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(
-				MapFunction<Tuple2<EV, T>, EV> mapper) {
-			this.mapper = mapper;
+		public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(EdgeJoinFunction<EV, T> mapper) {
+			this.edgeJoinFunction = mapper;
 		}
 
 		@Override
@@ -641,8 +719,7 @@ public class Graph<K, VV, EV> {
 					Edge<K, EV> edgesNext = edgesIterator.next();
 
 					collector.collect(new Edge<K, EV>(edgesNext.f0,
-							edgesNext.f1, mapper.map(new Tuple2<EV, T>(
-									edgesNext.f2, inputNext.f1))));
+							edgesNext.f1, edgeJoinFunction.edgeJoin(edgesNext.f2, inputNext.f1)));
 				}
 
 			} else {
@@ -654,22 +731,26 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Joins the edge DataSet with an input DataSet on the target key of the
-	 * edges and the first attribute of the input DataSet and applies a UDF on
-	 * the resulted values. Should the inputDataSet contain the same key more
-	 * than once, only the first value will be considered.
+	 * Joins the edge DataSet with an input Tuple2 DataSet and applies a user-defined transformation
+	 * on the values of the matched records.
+	 * The target ID of the edges input and the first field of the input DataSet are used as join keys.
 	 * 
 	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
+	 * The first field of the Tuple2 is used as the join key
+	 * and the second field is passed as a parameter to the transformation function. 
+	 * @param edgeJoinFunction the transformation function to apply.
+	 * The first parameter is the current edge value and the second parameter is the value
+	 * of the matched Tuple2 from the input DataSet.
+	 * @param <T> the type of the second field of the input Tuple2 DataSet.
+	 * @return a new Graph, where the edge values have been updated according to the
+	 * result of the edgeJoinFunction.
+	*/
 	public <T> Graph<K, VV, EV> joinWithEdgesOnTarget(DataSet<Tuple2<K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+			final EdgeJoinFunction<EV, T> edgeJoinFunction) {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(1).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(edgeJoinFunction));
 
 		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
 	}
@@ -798,20 +879,21 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges has access to the vertex value.
+	 * Groups by vertex and computes a GroupReduce transformation over the edge values of each vertex.
+	 * The edgesFunction applied on the edges has access to both the id and the value
+	 * of the grouping vertex.
+	 * 
+	 * For each vertex, the edgesFunction can iterate over all edges of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
 	 * 
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @return a dataset of a T
+	 * @param edgesFunction the group reduce function to apply to the neighboring edges of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
+	 * @param <T> the output type
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
-											EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 
 		switch (direction) {
 		case IN:
@@ -829,21 +911,22 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges has access to the vertex value.
-	 *
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
+	 * Groups by vertex and computes a GroupReduce transformation over the edge values of each vertex.
+	 * The edgesFunction applied on the edges has access to both the id and the value
+	 * of the grouping vertex.
+	 * 
+	 * For each vertex, the edgesFunction can iterate over all edges of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param edgesFunction the group reduce function to apply to the neighboring edges of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
+	 * @param <T> the output type
 	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
-											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+			EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
 
 		switch (direction) {
 			case IN:
@@ -861,20 +944,21 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges only has access to the vertex id (not the vertex value).
+	 * Groups by vertex and computes a GroupReduce transformation over the edge values of each vertex.
+	 * The edgesFunction applied on the edges only has access to the vertex id (not the vertex value)
+	 * of the grouping vertex.
 	 * 
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @return a dataset of T
+	 * For each vertex, the edgesFunction can iterate over all edges of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param edgesFunction the group reduce function to apply to the neighboring edges of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
+	 * @param <T> the output type
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
-											EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 
 		switch (direction) {
 		case IN:
@@ -894,21 +978,22 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges only has access to the vertex id (not the vertex value).
-	 *
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
+	 * Groups by vertex and computes a GroupReduce transformation over the edge values of each vertex.
+	 * The edgesFunction applied on the edges only has access to the vertex id (not the vertex value)
+	 * of the grouping vertex.
+	 * 
+	 * For each vertex, the edgesFunction can iterate over all edges of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param edgesFunction the group reduce function to apply to the neighboring edges of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
+	 * @param <T> the output type
 	 * @param typeInfo the explicit return type.
-	 * @return a dataset of T
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
-											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+			EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
 
 		switch (direction) {
 			case IN:
@@ -1515,18 +1600,22 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors has access to the vertex
-	 * value.
+	 * Groups by vertex and computes a GroupReduce transformation over the neighbors (both edges and vertices)
+	 * of each vertex. The neighborsFunction applied on the neighbors only has access to both the vertex id
+	 * and the vertex value of the grouping vertex.
 	 * 
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
+	 * For each vertex, the neighborsFunction can iterate over all neighbors of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param neighborsFunction the group reduce function to apply to the neighboring edges and vertices
+	 * of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
 	 * @param <T> the output type
-	 * @return a dataset of a T
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 		switch (direction) {
 		case IN:
 			// create <edge-sourceVertex> pairs
@@ -1558,19 +1647,23 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors has access to the vertex
-	 * value.
-	 *
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
+	 * Groups by vertex and computes a GroupReduce transformation over the neighbors (both edges and vertices)
+	 * of each vertex. The neighborsFunction applied on the neighbors only has access to both the vertex id
+	 * and the vertex value of the grouping vertex.
+	 * 
+	 * For each vertex, the neighborsFunction can iterate over all neighbors of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param neighborsFunction the group reduce function to apply to the neighboring edges and vertices
+	 * of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
 	 * @param <T> the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
+	 * @param typeInfo the explicit return type
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+			EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
 		switch (direction) {
 			case IN:
 				// create <edge-sourceVertex> pairs
@@ -1601,20 +1694,23 @@ public class Graph<K, VV, EV> {
 		}
 	}
 
-
 	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors only has access to the
-	 * vertex id (not the vertex value).
+	 * Groups by vertex and computes a GroupReduce transformation over the neighbors (both edges and vertices)
+	 * of each vertex. The neighborsFunction applied on the neighbors only has access to the vertex id
+	 * (not the vertex value) of the grouping vertex.
 	 * 
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
+	 * For each vertex, the neighborsFunction can iterate over all neighbors of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param neighborsFunction the group reduce function to apply to the neighboring edges and vertices
+	 * of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
 	 * @param <T> the output type
-	 * @return a dataset of a T
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 		switch (direction) {
 		case IN:
 			// create <edge-sourceVertex> pairs
@@ -1647,19 +1743,23 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors only has access to the
-	 * vertex id (not the vertex value).
-	 *
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
+	 * Groups by vertex and computes a GroupReduce transformation over the neighbors (both edges and vertices)
+	 * of each vertex. The neighborsFunction applied on the neighbors only has access to the vertex id
+	 * (not the vertex value) of the grouping vertex.
+	 * 
+	 * For each vertex, the neighborsFunction can iterate over all neighbors of this vertex
+	 * with the specified direction, and emit any number of output elements, including none.
+	 * 
+	 * @param neighborsFunction the group reduce function to apply to the neighboring edges and vertices
+	 * of each vertex.
+	 * @param direction the edge direction (in-, out-, all-).
 	 * @param <T> the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
+	 * @param typeInfo the explicit return type
+	 * @return a DataSet containing elements of type T
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+			EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
 		switch (direction) {
 			case IN:
 				// create <edge-sourceVertex> pairs
@@ -1842,16 +1942,21 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the neighbor values of each
-	 * vertex.
+	 * Compute a reduce transformation over the neighbors' vertex values of each vertex.
+	 * For each vertex, the transformation consecutively calls a
+	 * {@link ReduceNeighborsFunction} until only a single value for each vertex remains.
+	 * The {@link ReduceNeighborsFunction} combines a pair of neighbor vertex values
+	 * into one new value of the same type.
 	 *
-	 * @param reduceNeighborsFunction the function to apply to the neighborhood
+	 * @param reduceNeighborsFunction the reduce function to apply to the neighbors of each vertex.
 	 * @param direction the edge direction (in-, out-, all-)
-	 * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
+	 * @return a Dataset of Tuple2, with one tuple per vertex.
+	 * The first field of the Tuple2 is the vertex ID and the second field
+	 * is the aggregate value computed by the provided {@link ReduceNeighborsFunction}.
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public DataSet<Tuple2<K, VV>> reduceOnNeighbors(ReduceNeighborsFunction<VV> reduceNeighborsFunction,
-									EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 		switch (direction) {
 			case IN:
 				// create <vertex-source value> pairs
@@ -1900,17 +2005,20 @@ public class Graph<K, VV, EV> {
 	}
 
 	/**
-	 * Compute an aggregate over the edge values of each vertex.
+	 * Compute a reduce transformation over the edge values of each vertex.
+	 * For each vertex, the transformation consecutively calls a
+	 * {@link ReduceEdgesFunction} until only a single value for each edge remains.
+	 * The {@link ReduceEdgesFunction} combines two edge values into one new value of the same type.
 	 *
-	 * @param reduceEdgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @return a Dataset containing one value per vertex(vertex key, aggregate edge value)
+	 * @param reduceEdgesFunction the reduce function to apply to the neighbors of each vertex.
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @return a Dataset of Tuple2, with one tuple per vertex.
+	 * The first field of the Tuple2 is the vertex ID and the second field
+	 * is the aggregate value computed by the provided {@link ReduceEdgesFunction}.
 	 * @throws IllegalArgumentException
-	 */
+	*/
 	public DataSet<Tuple2<K, EV>> reduceOnEdges(ReduceEdgesFunction<EV> reduceEdgesFunction,
-								EdgeDirection direction) throws IllegalArgumentException {
+			EdgeDirection direction) throws IllegalArgumentException {
 
 		switch (direction) {
 			case IN:

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
index a21b23d..1a32204 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
@@ -36,5 +36,24 @@ import org.apache.flink.util.Collector;
  */
 public interface NeighborsFunction<K, VV, EV, O> extends Function, Serializable {
 
+	/**
+	 * This method is called per vertex and can iterate over all of its neighbors
+	 * with the specified direction.
+	 * <p>
+	 * If called with {@link EdgeDirection#OUT} the group will contain
+	 * the out-edges and neighboring vertices of the grouping vertex.
+	 * If called with {@link EdgeDirection#IN} the group will contain
+	 * the in-edges and neighboring vertices of the grouping vertex.
+	 * If called with {@link EdgeDirection#ALL} the group will contain
+	 * all edges and neighboring vertices of the grouping vertex.
+	 * <p>
+	 * The method can emit any number of output elements, including none.
+	 * 
+	 * @param neighbors the neighbors of the grouping vertex.
+	 * The first filed of each Tuple3 is the ID of the grouping vertex.
+	 * The second field is the neighboring edge, and the third field is the neighboring vertex.
+	 * @param out the collector to emit results to
+	 * @throws Exception
+	*/
 	void iterateNeighbors(Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
index fdf54fa..657238c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
@@ -36,5 +36,25 @@ import org.apache.flink.util.Collector;
  */
 public interface NeighborsFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
 
+	/**
+	 * This method is called per vertex and can iterate over all of its neighbors
+	 * with the specified direction.
+	 * <p>
+	 * If called with {@link EdgeDirection#OUT} the group will contain
+	 * the out-edges and neighboring vertices of the grouping vertex.
+	 * If called with {@link EdgeDirection#IN} the group will contain
+	 * the in-edges and neighboring vertices of the grouping vertex.
+	 * If called with {@link EdgeDirection#ALL} the group will contain
+	 * all edges and neighboring vertices of the grouping vertex.
+	 * <p>
+	 * The method can emit any number of output elements, including none.
+	 * 
+	 * @param vertex the grouping Vertex
+	 * @param neighbors the neighbors of the grouping vertex.
+	 * The first filed of each Tuple3 is the ID of the grouping vertex.
+	 * The second field is the neighboring edge, and the third field is the neighboring vertex.
+	 * @param out the collector to emit results to
+	 * @throws Exception
+	 */
 	void iterateNeighbors(Vertex<K, VV> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
index 84eec51..e7631a1 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
@@ -30,5 +30,14 @@ import java.io.Serializable;
  */
 public interface ReduceEdgesFunction<EV> extends Function, Serializable {
 
+	/**
+	 * It combines two neighboring edge values into one new value of the same type.
+	 * For each vertex, this function is consecutively called,
+	 * until only a single value for each edge remains.
+	 * 
+	 * @param firstEdgeValue the first neighboring edge value to combine
+	 * @param secondEdgeValue the second neighboring edge value to combine
+	 * @return the combined value of both input values
+	 */
 	EV reduceEdges(EV firstEdgeValue, EV secondEdgeValue);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
index fc5295d..5b423e2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
@@ -31,5 +31,14 @@ import java.io.Serializable;
  */
 public interface ReduceNeighborsFunction <VV> extends Function, Serializable {
 
+	/**
+	 * It combines two neighboring vertex values into one new value of the same type.
+	 * For each vertex, this function is consecutively called,
+	 * until only a single value for each vertex remains.
+	 * 
+	 * @param firstNeighborValue the first neighboring vertex value to combine
+	 * @param secondNeighborValue the second neighboring vertex value to combine
+	 * @return the combined value of both input values
+	 */
 	VV reduceNeighbors(VV firstNeighborValue, VV secondNeighborValue);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/640e63be/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/VertexJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/VertexJoinFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/VertexJoinFunction.java
new file mode 100644
index 0000000..a30d1a2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/VertexJoinFunction.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.graph;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+
+/**
+ * Interface to be implemented by the transformation function
+ * applied in {@link Graph#joinWithVertices(DataSet, VertexJoinFunction)} method.
+ *
+ * @param <VV> the vertex value type
+ * @param <T> the input value type
+ */
+public interface VertexJoinFunction<VV, T> extends Function, Serializable {
+
+	/**
+	 * Applies a transformation on the current vertex value
+	 * and the value of the matched tuple of the input DataSet.
+	 * 
+	 * @param vertexValue the current vertex value
+	 * @param inputValue the value of the matched Tuple2 input
+	 * @return the new vertex value
+	 */
+	VV vertexJoin(VV vertexValue, T inputValue) throws Exception;
+}


[07/10] flink git commit: [FLINK-2107] Add hash-based strategies for left and right outer joins.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
deleted file mode 100644
index 1795062..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
+++ /dev/null
@@ -1,766 +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.runtime.operators.hash;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-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.testutils.DiscardingOutputCollector;
-import org.apache.flink.runtime.operators.testutils.DummyInvokable;
-import org.apache.flink.runtime.operators.testutils.TestData;
-import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
-import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
-import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
-import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
-import org.apache.flink.runtime.operators.testutils.UnionIterator;
-import org.apache.flink.runtime.operators.testutils.types.IntPair;
-import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer;
-import org.apache.flink.types.NullKeyFieldException;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.flink.api.common.typeutils.GenericPairComparator;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-@SuppressWarnings({"serial", "deprecation"})
-public class NonReusingHashMatchIteratorITCase {
-	
-	private static final int MEMORY_SIZE = 16000000;		// total memory
-
-	private static final int INPUT_1_SIZE = 20000;
-	private static final int INPUT_2_SIZE = 1000;
-
-	private static final long SEED1 = 561349061987311L;
-	private static final long SEED2 = 231434613412342L;
-	
-	private final AbstractInvokable parentTask = new DummyInvokable();
-
-	private IOManager ioManager;
-	private MemoryManager memoryManager;
-	
-	private TypeSerializer<Tuple2<Integer, String>> recordSerializer;
-	private TypeComparator<Tuple2<Integer, String>> record1Comparator;
-	private TypeComparator<Tuple2<Integer, String>> record2Comparator;
-	private TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> recordPairComparator;
-	
-	private TypeSerializer<IntPair> pairSerializer;
-	private TypeComparator<IntPair> pairComparator;
-	private TypePairComparator<IntPair, Tuple2<Integer, String>> pairRecordPairComparator;
-	private TypePairComparator<Tuple2<Integer, String>, IntPair> recordPairPairComparator;
-
-
-	@SuppressWarnings("unchecked")
-	@Before
-	public void beforeTest() {
-		this.recordSerializer = TestData.getIntStringTupleSerializer();
-		
-		this.record1Comparator = TestData.getIntStringTupleComparator();
-		this.record2Comparator = TestData.getIntStringTupleComparator();
-		
-		this.recordPairComparator = new GenericPairComparator(record1Comparator, record2Comparator);
-		
-		this.pairSerializer = new IntPairSerializer();
-		this.pairComparator = new TestData.IntPairComparator();
-		this.pairRecordPairComparator = new IntPairTuplePairComparator();
-		this.recordPairPairComparator = new TupleIntPairPairComparator();
-		
-		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
-		this.ioManager = new IOManagerAsync();
-	}
-
-	@After
-	public void afterTest() {
-		if (this.ioManager != null) {
-			this.ioManager.shutdown();
-			if (!this.ioManager.isProperlyShutDown()) {
-				Assert.fail("I/O manager failed to properly shut down.");
-			}
-			this.ioManager = null;
-		}
-		
-		if (this.memoryManager != null) {
-			Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.",
-				this.memoryManager.verifyEmpty());
-			this.memoryManager.shutdown();
-			this.memoryManager = null;
-		}
-	}
-
-
-	@Test
-	public void testBuildFirst() {
-		try {
-			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<Tuple2<Integer, String>>();
-	
-			// reset the generators
-			generator1.reset();
-			generator2.reset();
-			input1.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			NonReusingBuildFirstHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new NonReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.recordSerializer, this.record1Comparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildFirstWithHighNumberOfCommonKeys()
-	{
-		// the size of the left and right inputs
-		final int INPUT_1_SIZE = 200;
-		final int INPUT_2_SIZE = 100;
-		
-		final int INPUT_1_DUPLICATES = 10;
-		final int INPUT_2_DUPLICATES = 2000;
-		final int DUPLICATE_KEY = 13;
-		
-		try {
-			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
-			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-			
-			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
-			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
-			
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			// re-create the whole thing for actual processing
-			
-			// reset the generators and iterators
-			generator1.reset();
-			generator2.reset();
-			const1Iter.reset();
-			const2Iter.reset();
-			gen1Iter.reset();
-			gen2Iter.reset();
-			
-			inList1.clear();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			inList2.clear();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-	
-			input1 = new UnionIterator<>(inList1);
-			input2 = new UnionIterator<>(inList2);
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			NonReusingBuildFirstHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new NonReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.recordSerializer, this.record1Comparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, 1.0, true);
-
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecond() {
-		try {
-			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			generator1.reset();
-			generator2.reset();
-			input1.reset();
-			input2.reset();
-	
-			// compare with iterator values			
-			NonReusingBuildSecondHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-				new NonReusingBuildSecondHashMatchIterator<>(
-					input1, input2, this.recordSerializer, this.record1Comparator, 
-					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
-
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecondWithHighNumberOfCommonKeys()
-	{
-		// the size of the left and right inputs
-		final int INPUT_1_SIZE = 200;
-		final int INPUT_2_SIZE = 100;
-		
-		final int INPUT_1_DUPLICATES = 10;
-		final int INPUT_2_DUPLICATES = 2000;
-		final int DUPLICATE_KEY = 13;
-		
-		try {
-			TupleGenerator generator1 = new TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
-			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-			
-			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
-			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
-			
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			// re-create the whole thing for actual processing
-			
-			// reset the generators and iterators
-			generator1.reset();
-			generator2.reset();
-			const1Iter.reset();
-			const2Iter.reset();
-			gen1Iter.reset();
-			gen2Iter.reset();
-			
-			inList1.clear();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			inList2.clear();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-	
-			input1 = new UnionIterator<>(inList1);
-			input2 = new UnionIterator<>(inList2);
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-
-			NonReusingBuildSecondHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-				new NonReusingBuildSecondHashMatchIterator<>(
-					input1, input2, this.recordSerializer, this.record1Comparator, 
-					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildFirstWithMixedDataTypes() {
-		try {
-			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
-			
-			final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = matchTupleIntPairValues(
-				collectIntPairData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<Tuple2<Integer, String>>();
-	
-			// reset the generators
-			input1 = new UniformIntPairGenerator(500, 40, false);
-			generator2.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			NonReusingBuildSecondHashMatchIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new NonReusingBuildSecondHashMatchIterator<>(
-						input1, input2, this.pairSerializer, this.pairComparator,
-						this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecondWithMixedDataTypes() {
-		try {
-			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
-			
-			final TupleGenerator generator2 = new TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = matchTupleIntPairValues(
-				collectIntPairData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			input1 = new UniformIntPairGenerator(500, 40, false);
-			generator2.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			NonReusingBuildFirstHashMatchIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new NonReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.pairSerializer, this.pairComparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                    Utilities
-	// --------------------------------------------------------------------------------------------
-
-	
-	
-	static Map<Integer, Collection<TupleMatch>> matchSecondTupleFields(
-			Map<Integer, Collection<String>> leftMap,
-			Map<Integer, Collection<String>> rightMap)
-	{
-		Map<Integer, Collection<TupleMatch>> map = new HashMap<>();
-
-		for (Integer key : leftMap.keySet()) {
-			Collection<String> leftValues = leftMap.get(key);
-			Collection<String> rightValues = rightMap.get(key);
-
-			if (rightValues == null) {
-				continue;
-			}
-
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<TupleMatch>());
-			}
-
-			Collection<TupleMatch> matchedValues = map.get(key);
-
-			for (String leftValue : leftValues) {
-				for (String rightValue : rightValues) {
-					matchedValues.add(new TupleMatch(leftValue, rightValue));
-				}
-			}
-		}
-
-		return map;
-	}
-	
-	static Map<Integer, Collection<TupleIntPairMatch>> matchTupleIntPairValues(
-		Map<Integer, Collection<Integer>> leftMap,
-		Map<Integer, Collection<String>> rightMap)
-	{
-		final Map<Integer, Collection<TupleIntPairMatch>> map = new HashMap<>();
-	
-		for (Integer i : leftMap.keySet()) {
-			
-			final Collection<Integer> leftValues = leftMap.get(i);
-			final Collection<String> rightValues = rightMap.get(i);
-	
-			if (rightValues == null) {
-				continue;
-			}
-	
-			if (!map.containsKey(i)) {
-				map.put(i, new ArrayList<TupleIntPairMatch>());
-			}
-	
-			final Collection<TupleIntPairMatch> matchedValues = map.get(i);
-	
-			for (Integer v : leftValues) {
-				for (String val : rightValues) {
-					matchedValues.add(new TupleIntPairMatch(v, val));
-				}
-			}
-		}
-	
-		return map;
-	}
-
-	
-	static Map<Integer, Collection<String>> collectTupleData(MutableObjectIterator<Tuple2<Integer, String>> iter)
-	throws Exception
-	{
-		Map<Integer, Collection<String>> map = new HashMap<>();
-		Tuple2<Integer, String> pair = new Tuple2<>();
-		
-		while ((pair = iter.next(pair)) != null) {
-
-			Integer key = pair.f0;
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<String>());
-			}
-
-			Collection<String> values = map.get(key);
-			values.add(pair.f1);
-		}
-
-		return map;
-	}
-	
-	static Map<Integer, Collection<Integer>> collectIntPairData(MutableObjectIterator<IntPair> iter)
-	throws Exception
-	{
-		Map<Integer, Collection<Integer>> map = new HashMap<Integer, Collection<Integer>>();
-		IntPair pair = new IntPair();
-		
-		while ((pair = iter.next(pair)) != null) {
-
-			final int key = pair.getKey();
-			final int value = pair.getValue();
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<Integer>());
-			}
-
-			Collection<Integer> values = map.get(key);
-			values.add(value);
-		}
-
-		return map;
-	}
-
-	/**
-	 * Private class used for storage of the expected matches in a hash-map.
-	 */
-	static class TupleMatch {
-		
-		private final String left;
-		private final String right;
-
-		public TupleMatch(String left, String right) {
-			this.left = left;
-			this.right = right;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			TupleMatch o = (TupleMatch) obj;
-			return this.left.equals(o.left) && this.right.equals(o.right);
-		}
-		
-		@Override
-		public int hashCode() {
-			return this.left.hashCode() ^ this.right.hashCode();
-		}
-
-		@Override
-		public String toString() {
-			return left + ", " + right;
-		}
-	}
-	
-	/**
-	 * Private class used for storage of the expected matches in a hash-map.
-	 */
-	static class TupleIntPairMatch
-	{
-		private final int left;
-		private final String right;
-
-		public TupleIntPairMatch(int left, String right) {
-			this.left = left;
-			this.right = new String(right);
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			TupleIntPairMatch o = (TupleIntPairMatch) obj;
-			return this.left == o.left && this.right.equals(o.right);
-		}
-		
-		@Override
-		public int hashCode() {
-			return this.left ^ this.right.hashCode();
-		}
-
-		@Override
-		public String toString() {
-			return left + ", " + right;
-		}
-	}
-	
-	static final class TupleMatchRemovingJoin implements FlatJoinFunction<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
-	{
-		private final Map<Integer, Collection<TupleMatch>> toRemoveFrom;
-		
-		protected TupleMatchRemovingJoin(Map<Integer, Collection<TupleMatch>> map) {
-			this.toRemoveFrom = map;
-		}
-		
-		@Override
-		public void join(Tuple2<Integer, String> rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
-		{
-			int key = rec1.f0;
-			String value1 = rec1.f1;
-			String value2 = rec2.f1;
-			//System.err.println("rec1 key = "+key+"  rec2 key= "+rec2.f0);
-			Collection<TupleMatch> matches = this.toRemoveFrom.get(key);
-			if (matches == null) {
-				Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected.");
-			}
-			
-			Assert.assertTrue("Produced match was not contained: " + key + " - " + value1 + ":" + value2,
-				matches.remove(new TupleMatch(value1, value2)));
-			
-			if (matches.isEmpty()) {
-				this.toRemoveFrom.remove(key);
-			}
-		}
-	}
-	
-	static final class TupleIntPairMatchRemovingMatcher implements FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>>
-	{
-		private final Map<Integer, Collection<TupleIntPairMatch>> toRemoveFrom;
-		
-		protected TupleIntPairMatchRemovingMatcher(Map<Integer, Collection<TupleIntPairMatch>> map) {
-			this.toRemoveFrom = map;
-		}
-		
-		@Override
-		public void join(IntPair rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
-		{
-			final int k = rec1.getKey();
-			final int v = rec1.getValue(); 
-			
-			final Integer key = rec2.f0;
-			final String value = rec2.f1;
-
-			Assert.assertTrue("Key does not match for matching IntPair Tuple combination.", k == key);
-			
-			Collection<TupleIntPairMatch> matches = this.toRemoveFrom.get(key);
-			if (matches == null) {
-				Assert.fail("Match " + key + " - " + v + ":" + value + " is unexpected.");
-			}
-			
-			Assert.assertTrue("Produced match was not contained: " + key + " - " + v + ":" + value,
-				matches.remove(new TupleIntPairMatch(v, value)));
-			
-			if (matches.isEmpty()) {
-				this.toRemoveFrom.remove(key);
-			}
-		}
-	}
-	
-	static final class IntPairTuplePairComparator extends TypePairComparator<IntPair, Tuple2<Integer, String>>
-	{
-		private int reference;
-		
-		@Override
-		public void setReference(IntPair reference) {
-			this.reference = reference.getKey();	
-		}
-
-		@Override
-		public boolean equalToReference(Tuple2<Integer, String> candidate) {
-			try {
-				return candidate.f0 == this.reference;
-			} catch (NullPointerException npex) {
-				throw new NullKeyFieldException();
-			}
-		}
-
-		@Override
-		public int compareToReference(Tuple2<Integer, String> candidate) {
-			try {
-				return candidate.f0 - this.reference;
-			} catch (NullPointerException npex) {
-				throw new NullKeyFieldException();
-			}
-		}
-	}
-	
-	static final class TupleIntPairPairComparator extends TypePairComparator<Tuple2<Integer, String>, IntPair>
-	{
-		private int reference;
-		
-		@Override
-		public void setReference(Tuple2<Integer, String> reference) {
-			this.reference = reference.f0;
-		}
-
-		@Override
-		public boolean equalToReference(IntPair candidate) {
-			return this.reference == candidate.getKey();
-		}
-
-		@Override
-		public int compareToReference(IntPair candidate) {
-			return candidate.getKey() - this.reference;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
index 5a4fc6a..a885e6b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
@@ -30,8 +30,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
-import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase.TupleMatch;
-import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase.TupleMatchRemovingJoin;
+import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatch;
+import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatchRemovingJoin;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
@@ -204,7 +204,7 @@ public class NonReusingReOpenableHashTableITCase {
 
 	private void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TupleGenerator bgen, TupleGenerator pgen) throws Exception {
 		// collect expected data
-		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = NonReusingHashMatchIteratorITCase.matchSecondTupleFields(NonReusingHashMatchIteratorITCase.collectTupleData(buildInput), NonReusingHashMatchIteratorITCase.collectTupleData(probeInput));
+		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = NonReusingHashJoinIteratorITCase.joinTuples(NonReusingHashJoinIteratorITCase.collectTupleData(buildInput), NonReusingHashJoinIteratorITCase.collectTupleData(probeInput));
 
 		final List<Map<Integer, Collection<TupleMatch>>> expectedNMatchesMapList = new ArrayList<>(NUM_PROBES);
 		final FlatJoinFunction[] nMatcher = new TupleMatchRemovingJoin[NUM_PROBES];
@@ -225,11 +225,11 @@ public class NonReusingReOpenableHashTableITCase {
 		probeInput.reset();
 
 		// compare with iterator values
-		NonReusingBuildFirstReOpenableHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-				new NonReusingBuildFirstReOpenableHashMatchIterator<>(
+		NonReusingBuildFirstReOpenableHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+				new NonReusingBuildFirstReOpenableHashJoinIterator<>(
 						buildInput, probeInput, this.recordSerializer, this.record1Comparator,
 					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
 
 		iterator.open();
 		// do first join with both inputs

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java
new file mode 100644
index 0000000..87707a4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java
@@ -0,0 +1,709 @@
+/*
+ * 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.runtime.operators.hash;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+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.api.java.tuple.Tuple2;
+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.hash.NonReusingHashJoinIteratorITCase.TupleMatch;
+import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleIntPairMatch;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.TestData;
+import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
+import org.apache.flink.runtime.operators.testutils.UnionIterator;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
+import org.apache.flink.runtime.operators.testutils.types.IntPair;
+import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer;
+import org.apache.flink.types.NullKeyFieldException;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.joinTuples;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.leftOuterJoinTuples;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.rightOuterJoinTuples;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.joinIntPairs;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.collectTupleData;
+import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.collectIntPairData;
+
+@SuppressWarnings({"serial", "deprecation"})
+public class ReusingHashJoinIteratorITCase {
+	
+	private static final int MEMORY_SIZE = 16000000;		// total memory
+
+	private static final int INPUT_1_SIZE = 20000;
+	private static final int INPUT_2_SIZE = 1000;
+
+	private static final long SEED1 = 561349061987311L;
+	private static final long SEED2 = 231434613412342L;
+	
+	private final AbstractInvokable parentTask = new DummyInvokable();
+
+	private IOManager ioManager;
+	private MemoryManager memoryManager;
+	
+	private TypeSerializer<Tuple2<Integer, String>> recordSerializer;
+	private TypeComparator<Tuple2<Integer, String>> record1Comparator;
+	private TypeComparator<Tuple2<Integer, String>> record2Comparator;
+	private TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> recordPairComparator;
+	
+	private TypeSerializer<IntPair> pairSerializer;
+	private TypeComparator<IntPair> pairComparator;
+	private TypePairComparator<IntPair, Tuple2<Integer, String>> pairRecordPairComparator;
+	private TypePairComparator<Tuple2<Integer, String>, IntPair> recordPairPairComparator;
+
+
+	@SuppressWarnings("unchecked")
+	@Before
+	public void beforeTest() {
+		this.recordSerializer = TestData.getIntStringTupleSerializer();
+		
+		this.record1Comparator = TestData.getIntStringTupleComparator();
+		this.record2Comparator = TestData.getIntStringTupleComparator();
+		
+		this.recordPairComparator = new GenericPairComparator(this.record1Comparator, this.record2Comparator);
+		
+		this.pairSerializer = new IntPairSerializer();
+		this.pairComparator = new TestData.IntPairComparator();
+		this.pairRecordPairComparator = new IntPairTuplePairComparator();
+		this.recordPairPairComparator = new TupleIntPairPairComparator();
+		
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
+		this.ioManager = new IOManagerAsync();
+	}
+
+	@After
+	public void afterTest() {
+		if (this.ioManager != null) {
+			this.ioManager.shutdown();
+			if (!this.ioManager.isProperlyShutDown()) {
+				Assert.fail("I/O manager failed to properly shut down.");
+			}
+			this.ioManager = null;
+		}
+		
+		if (this.memoryManager != null) {
+			Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.",
+				this.memoryManager.verifyEmpty());
+			this.memoryManager.shutdown();
+			this.memoryManager = null;
+		}
+	}
+
+
+	@Test
+	public void testBuildFirst() {
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			ReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.recordSerializer, this.record1Comparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+						this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildFirstWithHighNumberOfCommonKeys()
+	{
+		// the size of the left and right inputs
+		final int INPUT_1_SIZE = 200;
+		final int INPUT_2_SIZE = 100;
+		
+		final int INPUT_1_DUPLICATES = 10;
+		final int INPUT_2_DUPLICATES = 2000;
+		final int DUPLICATE_KEY = 13;
+		
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
+			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+			
+			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
+			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
+			
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			// re-create the whole thing for actual processing
+			
+			// reset the generators and iterators
+			generator1.reset();
+			generator2.reset();
+			const1Iter.reset();
+			const2Iter.reset();
+			gen1Iter.reset();
+			gen2Iter.reset();
+			
+			inList1.clear();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			inList2.clear();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+	
+			input1 = new UnionIterator<>(inList1);
+			input2 = new UnionIterator<>(inList2);
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			ReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.recordSerializer, this.record1Comparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+						this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecond() {
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+	
+			// compare with iterator values			
+			ReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>,Tuple2<Integer, String> ,Tuple2<Integer, String> > iterator =
+				new ReusingBuildSecondHashJoinIterator<>(
+					input1, input2, this.recordSerializer, this.record1Comparator, 
+					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecondWithHighNumberOfCommonKeys()
+	{
+		// the size of the left and right inputs
+		final int INPUT_1_SIZE = 200;
+		final int INPUT_2_SIZE = 100;
+		
+		final int INPUT_1_DUPLICATES = 10;
+		final int INPUT_2_DUPLICATES = 2000;
+		final int DUPLICATE_KEY = 13;
+		
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			
+			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
+			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+			
+			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
+			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
+			
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = joinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+			
+			// re-create the whole thing for actual processing
+			
+			// reset the generators and iterators
+			generator1.reset();
+			generator2.reset();
+			const1Iter.reset();
+			const2Iter.reset();
+			gen1Iter.reset();
+			gen2Iter.reset();
+			
+			inList1.clear();
+			inList1.add(gen1Iter);
+			inList1.add(const1Iter);
+			
+			inList2.clear();
+			inList2.add(gen2Iter);
+			inList2.add(const2Iter);
+	
+			input1 = new UnionIterator<>(inList1);
+			input2 = new UnionIterator<>(inList2);
+			
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			ReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+				new ReusingBuildSecondHashJoinIterator<>(
+					input1, input2, this.recordSerializer, this.record1Comparator, 
+					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+					this.memoryManager, ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildFirstWithMixedDataTypes() {
+		try {
+			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
+			
+			final TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = joinIntPairs(
+					collectIntPairData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			input1 = new UniformIntPairGenerator(500, 40, false);
+			generator2.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			ReusingBuildSecondHashJoinIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildSecondHashJoinIterator<>(
+						input1, input2, this.pairSerializer, this.pairComparator,
+						this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator,
+						this.memoryManager, this.ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBuildSecondWithMixedDataTypes() {
+		try {
+			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
+			
+			final TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+			
+			// collect expected data
+			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = joinIntPairs(
+					collectIntPairData(input1),
+					collectTupleData(input2));
+			
+			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+	
+			// reset the generators
+			input1 = new UniformIntPairGenerator(500, 40, false);
+			generator2.reset();
+			input2.reset();
+	
+			// compare with iterator values
+			ReusingBuildFirstHashJoinIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildFirstHashJoinIterator<>(
+						input1, input2, this.pairSerializer, this.pairComparator, 
+						this.recordSerializer, this.record2Comparator, this.recordPairPairComparator,
+						this.memoryManager, this.ioManager, this.parentTask, 1.0, false, true);
+			
+			iterator.open();
+			
+			while (iterator.callWithNextKey(matcher, collector));
+			
+			iterator.close();
+	
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBuildFirstJoinWithEmptyBuild() {
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 1000, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = rightOuterJoinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+
+			// compare with iterator values
+			ReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildFirstHashJoinIterator<>(
+							input1, input2, this.recordSerializer, this.record1Comparator,
+							this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+							this.memoryManager, ioManager, this.parentTask, 1.0, true, false);
+
+			iterator.open();
+
+			while (iterator.callWithNextKey(matcher, collector));
+
+			iterator.close();
+
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBuildSecondJoinWithEmptyBuild() {
+		try {
+			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 1000, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+
+			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
+			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
+
+			// collect expected data
+			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = leftOuterJoinTuples(
+					collectTupleData(input1),
+					collectTupleData(input2));
+
+			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
+
+			// reset the generators
+			generator1.reset();
+			generator2.reset();
+			input1.reset();
+			input2.reset();
+
+			// compare with iterator values
+			ReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
+					new ReusingBuildSecondHashJoinIterator<>(
+							input1, input2, this.recordSerializer, this.record1Comparator,
+							this.recordSerializer, this.record2Comparator, this.recordPairComparator,
+							this.memoryManager, ioManager, this.parentTask, 1.0, true, false);
+
+			iterator.open();
+
+			while (iterator.callWithNextKey(matcher, collector));
+
+			iterator.close();
+
+			// assert that each expected match was seen
+			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
+				if (!entry.getValue().isEmpty()) {
+					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	
+	// --------------------------------------------------------------------------------------------
+	//                                    Utilities
+	// --------------------------------------------------------------------------------------------
+
+
+	static final class TupleMatchRemovingJoin implements FlatJoinFunction<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
+	{
+		private final Map<Integer, Collection<TupleMatch>> toRemoveFrom;
+		
+		protected TupleMatchRemovingJoin(Map<Integer, Collection<TupleMatch>> map) {
+			this.toRemoveFrom = map;
+		}
+		
+		@Override
+		public void join(Tuple2<Integer, String> rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
+		{
+			Integer key = rec1 != null ? rec1.f0 : rec2.f0;
+			String value1 = rec1 != null ? rec1.f1 : null;
+			String value2 = rec2 != null ? rec2.f1 : null;
+			//System.err.println("rec1 key = "+key+"  rec2 key= "+rec2.getField(0, TestData.Key.class));
+			Collection<TupleMatch> matches = this.toRemoveFrom.get(key);
+			if (matches == null) {
+				Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected.");
+			}
+			
+			Assert.assertTrue("Produced match was not contained: " + key + " - " + value1 + ":" + value2,
+				matches.remove(new TupleMatch(value1, value2)));
+			
+			if (matches.isEmpty()) {
+				this.toRemoveFrom.remove(key);
+			}
+		}
+	}
+	
+	static final class TupleIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>>
+	{
+		private final Map<Integer, Collection<TupleIntPairMatch>> toRemoveFrom;
+		
+		protected TupleIntPairMatchRemovingMatcher(Map<Integer, Collection<TupleIntPairMatch>> map) {
+			this.toRemoveFrom = map;
+		}
+		
+		@Override
+		public void join(IntPair rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
+		{
+			final int k = rec1.getKey();
+			final int v = rec1.getValue(); 
+			
+			final Integer key = rec2.f0;
+			final String value = rec2.f1;
+			
+			Assert.assertTrue("Key does not match for matching IntPair Tuple combination.", k == key); 
+			
+			Collection<TupleIntPairMatch> matches = this.toRemoveFrom.get(key);
+			if (matches == null) {
+				Assert.fail("Match " + key + " - " + v + ":" + value + " is unexpected.");
+			}
+			
+			Assert.assertTrue("Produced match was not contained: " + key + " - " + v + ":" + value,
+				matches.remove(new TupleIntPairMatch(v, value)));
+			
+			if (matches.isEmpty()) {
+				this.toRemoveFrom.remove(key);
+			}
+		}
+	}
+	
+	static final class IntPairTuplePairComparator extends TypePairComparator<IntPair, Tuple2<Integer, String>>
+	{
+		private int reference;
+		
+		@Override
+		public void setReference(IntPair reference) {
+			this.reference = reference.getKey();	
+		}
+
+		@Override
+		public boolean equalToReference(Tuple2<Integer, String> candidate) {
+			try {
+				return candidate.f0 == this.reference;
+			} catch (NullPointerException npex) {
+				throw new NullKeyFieldException();
+			}
+		}
+
+		@Override
+		public int compareToReference(Tuple2<Integer, String> candidate) {
+			try {
+				return candidate.f0 - this.reference;
+			} catch (NullPointerException npex) {
+				throw new NullKeyFieldException();
+			}
+		}
+	}
+	
+	static final class TupleIntPairPairComparator extends TypePairComparator<Tuple2<Integer, String>, IntPair>
+	{
+		private int reference;
+		
+		@Override
+		public void setReference(Tuple2<Integer, String> reference) {
+			this.reference = reference.f0;
+		}
+
+		@Override
+		public boolean equalToReference(IntPair candidate) {
+			return this.reference == candidate.getKey();
+		}
+
+		@Override
+		public int compareToReference(IntPair candidate) {
+			return candidate.getKey() - this.reference;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5671c77c/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
deleted file mode 100644
index 12f4a32..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
+++ /dev/null
@@ -1,768 +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.runtime.operators.hash;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-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.api.java.tuple.Tuple2;
-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.testutils.DiscardingOutputCollector;
-import org.apache.flink.runtime.operators.testutils.DummyInvokable;
-import org.apache.flink.runtime.operators.testutils.TestData;
-import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
-import org.apache.flink.runtime.operators.testutils.UnionIterator;
-import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
-import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
-import org.apache.flink.runtime.operators.testutils.types.IntPair;
-import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer;
-import org.apache.flink.types.NullKeyFieldException;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-@SuppressWarnings({"serial", "deprecation"})
-public class ReusingHashMatchIteratorITCase {
-	
-	private static final int MEMORY_SIZE = 16000000;		// total memory
-
-	private static final int INPUT_1_SIZE = 20000;
-	private static final int INPUT_2_SIZE = 1000;
-
-	private static final long SEED1 = 561349061987311L;
-	private static final long SEED2 = 231434613412342L;
-	
-	private final AbstractInvokable parentTask = new DummyInvokable();
-
-	private IOManager ioManager;
-	private MemoryManager memoryManager;
-	
-	private TypeSerializer<Tuple2<Integer, String>> recordSerializer;
-	private TypeComparator<Tuple2<Integer, String>> record1Comparator;
-	private TypeComparator<Tuple2<Integer, String>> record2Comparator;
-	private TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> recordPairComparator;
-	
-	private TypeSerializer<IntPair> pairSerializer;
-	private TypeComparator<IntPair> pairComparator;
-	private TypePairComparator<IntPair, Tuple2<Integer, String>> pairRecordPairComparator;
-	private TypePairComparator<Tuple2<Integer, String>, IntPair> recordPairPairComparator;
-
-
-	@SuppressWarnings("unchecked")
-	@Before
-	public void beforeTest() {
-		this.recordSerializer = TestData.getIntStringTupleSerializer();
-		
-		this.record1Comparator = TestData.getIntStringTupleComparator();
-		this.record2Comparator = TestData.getIntStringTupleComparator();
-		
-		this.recordPairComparator = new GenericPairComparator(this.record1Comparator, this.record2Comparator);
-		
-		this.pairSerializer = new IntPairSerializer();
-		this.pairComparator = new TestData.IntPairComparator();
-		this.pairRecordPairComparator = new IntPairTuplePairComparator();
-		this.recordPairPairComparator = new TupleIntPairPairComparator();
-		
-		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
-		this.ioManager = new IOManagerAsync();
-	}
-
-	@After
-	public void afterTest() {
-		if (this.ioManager != null) {
-			this.ioManager.shutdown();
-			if (!this.ioManager.isProperlyShutDown()) {
-				Assert.fail("I/O manager failed to properly shut down.");
-			}
-			this.ioManager = null;
-		}
-		
-		if (this.memoryManager != null) {
-			Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.",
-				this.memoryManager.verifyEmpty());
-			this.memoryManager.shutdown();
-			this.memoryManager = null;
-		}
-	}
-
-
-	@Test
-	public void testBuildFirst() {
-		try {
-			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			generator1.reset();
-			generator2.reset();
-			input1.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			ReusingBuildFirstHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.recordSerializer, this.record1Comparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildFirstWithHighNumberOfCommonKeys()
-	{
-		// the size of the left and right inputs
-		final int INPUT_1_SIZE = 200;
-		final int INPUT_2_SIZE = 100;
-		
-		final int INPUT_1_DUPLICATES = 10;
-		final int INPUT_2_DUPLICATES = 2000;
-		final int DUPLICATE_KEY = 13;
-		
-		try {
-			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
-			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-			
-			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
-			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
-			
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			// re-create the whole thing for actual processing
-			
-			// reset the generators and iterators
-			generator1.reset();
-			generator2.reset();
-			const1Iter.reset();
-			const2Iter.reset();
-			gen1Iter.reset();
-			gen2Iter.reset();
-			
-			inList1.clear();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			inList2.clear();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-	
-			input1 = new UnionIterator<Tuple2<Integer, String>>(inList1);
-			input2 = new UnionIterator<Tuple2<Integer, String>>(inList2);
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			ReusingBuildFirstHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.recordSerializer, this.record1Comparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, 1.0, true);
-
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecond() {
-		try {
-			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator input1 = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			generator1.reset();
-			generator2.reset();
-			input1.reset();
-			input2.reset();
-	
-			// compare with iterator values			
-			ReusingBuildSecondHashMatchIterator<Tuple2<Integer, String>,Tuple2<Integer, String> ,Tuple2<Integer, String> > iterator =
-				new ReusingBuildSecondHashMatchIterator<>(
-					input1, input2, this.recordSerializer, this.record1Comparator, 
-					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
-
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecondWithHighNumberOfCommonKeys()
-	{
-		// the size of the left and right inputs
-		final int INPUT_1_SIZE = 200;
-		final int INPUT_2_SIZE = 100;
-		
-		final int INPUT_1_DUPLICATES = 10;
-		final int INPUT_2_DUPLICATES = 2000;
-		final int DUPLICATE_KEY = 13;
-		
-		try {
-			TestData.TupleGenerator generator1 = new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			
-			final TestData.TupleGeneratorIterator gen1Iter = new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
-			final TestData.TupleGeneratorIterator gen2Iter = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			final TestData.TupleConstantValueIterator const1Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
-			final TestData.TupleConstantValueIterator const2Iter = new TestData.TupleConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-			
-			MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
-			MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);
-			
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleMatch>> expectedMatchesMap = matchSecondTupleFields(
-				collectTupleData(input1),
-				collectTupleData(input2));
-			
-			// re-create the whole thing for actual processing
-			
-			// reset the generators and iterators
-			generator1.reset();
-			generator2.reset();
-			const1Iter.reset();
-			const2Iter.reset();
-			gen1Iter.reset();
-			gen2Iter.reset();
-			
-			inList1.clear();
-			inList1.add(gen1Iter);
-			inList1.add(const1Iter);
-			
-			inList2.clear();
-			inList2.add(gen2Iter);
-			inList2.add(const2Iter);
-	
-			input1 = new UnionIterator<>(inList1);
-			input2 = new UnionIterator<>(inList2);
-			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			ReusingBuildSecondHashMatchIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-				new ReusingBuildSecondHashMatchIterator<>(
-					input1, input2, this.recordSerializer, this.record1Comparator, 
-					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildFirstWithMixedDataTypes() {
-		try {
-			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
-			
-			final TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = matchTupleIntPairValues(
-				collectIntPairData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			input1 = new UniformIntPairGenerator(500, 40, false);
-			generator2.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			ReusingBuildSecondHashMatchIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildSecondHashMatchIterator<>(
-						input1, input2, this.pairSerializer, this.pairComparator,
-						this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBuildSecondWithMixedDataTypes() {
-		try {
-			MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);
-			
-			final TestData.TupleGenerator generator2 = new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
-			final TestData.TupleGeneratorIterator input2 = new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);
-			
-			// collect expected data
-			final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap = matchTupleIntPairValues(
-				collectIntPairData(input1),
-				collectTupleData(input2));
-			
-			final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher = new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
-			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
-	
-			// reset the generators
-			input1 = new UniformIntPairGenerator(500, 40, false);
-			generator2.reset();
-			input2.reset();
-	
-			// compare with iterator values
-			ReusingBuildFirstHashMatchIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
-					new ReusingBuildFirstHashMatchIterator<>(
-						input1, input2, this.pairSerializer, this.pairComparator, 
-						this.recordSerializer, this.record2Comparator, this.recordPairPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, 1.0, true);
-			
-			iterator.open();
-			
-			while (iterator.callWithNextKey(matcher, collector));
-			
-			iterator.close();
-	
-			// assert that each expected match was seen
-			for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
-				if (!entry.getValue().isEmpty()) {
-					Assert.fail("Collection for key " + entry.getKey() + " is not empty");
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                    Utilities
-	// --------------------------------------------------------------------------------------------
-
-	
-	
-	static Map<Integer, Collection<TupleMatch>> matchSecondTupleFields(
-			Map<Integer, Collection<String>> leftMap,
-			Map<Integer, Collection<String>> rightMap)
-	{
-		Map<Integer, Collection<TupleMatch>> map = new HashMap<>();
-
-		for (Integer key : leftMap.keySet()) {
-			Collection<String> leftValues = leftMap.get(key);
-			Collection<String> rightValues = rightMap.get(key);
-
-			if (rightValues == null) {
-				continue;
-			}
-
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<TupleMatch>());
-			}
-
-			Collection<TupleMatch> matchedValues = map.get(key);
-
-			for (String leftValue : leftValues) {
-				for (String rightValue : rightValues) {
-					matchedValues.add(new TupleMatch(leftValue, rightValue));
-				}
-			}
-		}
-
-		return map;
-	}
-	
-	static Map<Integer, Collection<TupleIntPairMatch>> matchTupleIntPairValues(
-		Map<Integer, Collection<Integer>> leftMap,
-		Map<Integer, Collection<String>> rightMap)
-	{
-		final Map<Integer, Collection<TupleIntPairMatch>> map = new HashMap<>();
-	
-		for (Integer i : leftMap.keySet()) {
-			
-			final Integer key = new Integer(i.intValue());
-			
-			final Collection<Integer> leftValues = leftMap.get(i);
-			final Collection<String> rightValues = rightMap.get(key);
-	
-			if (rightValues == null) {
-				continue;
-			}
-	
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<TupleIntPairMatch>());
-			}
-	
-			final Collection<TupleIntPairMatch> matchedValues = map.get(key);
-	
-			for (Integer v : leftValues) {
-				for (String val : rightValues) {
-					matchedValues.add(new TupleIntPairMatch(v, val));
-				}
-			}
-		}
-	
-		return map;
-	}
-
-	
-	static Map<Integer, Collection<String>> collectTupleData(MutableObjectIterator<Tuple2<Integer, String>> iter)
-	throws Exception
-	{
-		Map<Integer, Collection<String>> map = new HashMap<>();
-		Tuple2<Integer, String> pair = new Tuple2<>();
-		
-		while ((pair = iter.next(pair)) != null) {
-
-			Integer key = pair.f0;
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<String>());
-			}
-
-			Collection<String> values = map.get(key);
-			values.add(pair.f1);
-		}
-
-		return map;
-	}
-	
-	static Map<Integer, Collection<Integer>> collectIntPairData(MutableObjectIterator<IntPair> iter)
-	throws Exception
-	{
-		Map<Integer, Collection<Integer>> map = new HashMap<>();
-		IntPair pair = new IntPair();
-		
-		while ((pair = iter.next(pair)) != null) {
-
-			final int key = pair.getKey();
-			final int value = pair.getValue();
-			if (!map.containsKey(key)) {
-				map.put(key, new ArrayList<Integer>());
-			}
-
-			Collection<Integer> values = map.get(key);
-			values.add(value);
-		}
-
-		return map;
-	}
-
-	/**
-	 * Private class used for storage of the expected matches in a hash-map.
-	 */
-	static class TupleMatch {
-		
-		private final String left;
-		private final String right;
-
-		public TupleMatch(String left, String right) {
-			this.left = left;
-			this.right = right;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			TupleMatch o = (TupleMatch) obj;
-			return this.left.equals(o.left) && this.right.equals(o.right);
-		}
-		
-		@Override
-		public int hashCode() {
-			return this.left.hashCode() ^ this.right.hashCode();
-		}
-
-		@Override
-		public String toString() {
-			return left + ", " + right;
-		}
-	}
-	
-	/**
-	 * Private class used for storage of the expected matches in a hash-map.
-	 */
-	static class TupleIntPairMatch
-	{
-		private final int left;
-		private final String right;
-
-		public TupleIntPairMatch(int left, String right) {
-			this.left = left;
-			this.right = right;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			TupleIntPairMatch o = (TupleIntPairMatch) obj;
-			return this.left == o.left && this.right.equals(o.right);
-		}
-		
-		@Override
-		public int hashCode() {
-			return this.left ^ this.right.hashCode();
-		}
-
-		@Override
-		public String toString() {
-			return left + ", " + right;
-		}
-	}
-	
-	static final class TupleMatchRemovingJoin implements FlatJoinFunction<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
-	{
-		private final Map<Integer, Collection<TupleMatch>> toRemoveFrom;
-		
-		protected TupleMatchRemovingJoin(Map<Integer, Collection<TupleMatch>> map) {
-			this.toRemoveFrom = map;
-		}
-		
-		@Override
-		public void join(Tuple2<Integer, String> rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
-		{
-			Integer key = rec1.f0;
-			String value1 = rec1.f1;
-			String value2 = rec2.f1;
-			//System.err.println("rec1 key = "+key+"  rec2 key= "+rec2.getField(0, TestData.Key.class));
-			Collection<TupleMatch> matches = this.toRemoveFrom.get(key);
-			if (matches == null) {
-				Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected.");
-			}
-			
-			Assert.assertTrue("Produced match was not contained: " + key + " - " + value1 + ":" + value2,
-				matches.remove(new TupleMatch(value1, value2)));
-			
-			if (matches.isEmpty()) {
-				this.toRemoveFrom.remove(key);
-			}
-		}
-	}
-	
-	static final class TupleIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>>
-	{
-		private final Map<Integer, Collection<TupleIntPairMatch>> toRemoveFrom;
-		
-		protected TupleIntPairMatchRemovingMatcher(Map<Integer, Collection<TupleIntPairMatch>> map) {
-			this.toRemoveFrom = map;
-		}
-		
-		@Override
-		public void join(IntPair rec1, Tuple2<Integer, String> rec2, Collector<Tuple2<Integer, String>> out) throws Exception
-		{
-			final int k = rec1.getKey();
-			final int v = rec1.getValue(); 
-			
-			final Integer key = rec2.f0;
-			final String value = rec2.f1;
-			
-			Assert.assertTrue("Key does not match for matching IntPair Tuple combination.", k == key); 
-			
-			Collection<TupleIntPairMatch> matches = this.toRemoveFrom.get(key);
-			if (matches == null) {
-				Assert.fail("Match " + key + " - " + v + ":" + value + " is unexpected.");
-			}
-			
-			Assert.assertTrue("Produced match was not contained: " + key + " - " + v + ":" + value,
-				matches.remove(new TupleIntPairMatch(v, value)));
-			
-			if (matches.isEmpty()) {
-				this.toRemoveFrom.remove(key);
-			}
-		}
-	}
-	
-	static final class IntPairTuplePairComparator extends TypePairComparator<IntPair, Tuple2<Integer, String>>
-	{
-		private int reference;
-		
-		@Override
-		public void setReference(IntPair reference) {
-			this.reference = reference.getKey();	
-		}
-
-		@Override
-		public boolean equalToReference(Tuple2<Integer, String> candidate) {
-			try {
-				return candidate.f0 == this.reference;
-			} catch (NullPointerException npex) {
-				throw new NullKeyFieldException();
-			}
-		}
-
-		@Override
-		public int compareToReference(Tuple2<Integer, String> candidate) {
-			try {
-				return candidate.f0 - this.reference;
-			} catch (NullPointerException npex) {
-				throw new NullKeyFieldException();
-			}
-		}
-	}
-	
-	static final class TupleIntPairPairComparator extends TypePairComparator<Tuple2<Integer, String>, IntPair>
-	{
-		private int reference;
-		
-		@Override
-		public void setReference(Tuple2<Integer, String> reference) {
-			this.reference = reference.f0;
-		}
-
-		@Override
-		public boolean equalToReference(IntPair candidate) {
-			return this.reference == candidate.getKey();
-		}
-
-		@Override
-		public int compareToReference(IntPair candidate) {
-			return candidate.getKey() - this.reference;
-		}
-	}
-}