You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2017/02/20 00:55:15 UTC

[01/19] flink git commit: [FLINK-5817] [tests] Use TemporaryFold to create temp files and folds for test

Repository: flink
Updated Branches:
  refs/heads/master 70475b367 -> f113d7945


[FLINK-5817] [tests] Use TemporaryFold to create temp files and folds for test

This closes #3341


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

Branch: refs/heads/master
Commit: 709fa1d95b7dbbcfdd1124de7d6e073834ca75cf
Parents: d05fc37
Author: wenlong.lwl <we...@alibaba-inc.com>
Authored: Fri Feb 17 17:14:54 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 .../FileCacheDeleteValidationTest.java          | 12 +++--
 .../io/disk/iomanager/IOManagerTest.java        | 11 ++--
 .../flink/test/util/AbstractTestBase.java       | 53 ++++++--------------
 3 files changed, 31 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/709fa1d9/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
index 4db0d93..4dca3db 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.filecache;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.concurrent.Future;
 
 import org.apache.flink.core.fs.Path;
@@ -27,10 +28,12 @@ import org.apache.flink.api.common.JobID;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
+import org.junit.rules.TemporaryFolder;
 
 import static org.junit.Assert.fail;
 import static org.junit.Assert.assertTrue;
@@ -56,12 +59,15 @@ public class FileCacheDeleteValidationTest {
 		+ "Da flammt ein blitzendes Verheeren Dem Pfade vor des Donnerschlags. Doch\n"
 		+ "deine Boten, Herr, verehren Das sanfte Wandeln deines Tags.\n";
 
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
 	private FileCache fileCache;
 	private File f;
 	
 	@Before
-	public void setup() {
-		String[] tmpDirectories = System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator);
+	public void setup() throws IOException {
+		String[] tmpDirectories = new String[]{temporaryFolder.newFolder().getAbsolutePath()};
 		try {
 			fileCache = new FileCache(tmpDirectories);
 		}
@@ -70,7 +76,7 @@ public class FileCacheDeleteValidationTest {
 			fail("Cannot create FileCache: " + e.getMessage());
 		}
 		
-		f = new File(System.getProperty("java.io.tmpdir"), "cacheFile");
+		f = temporaryFolder.newFile("cacheFile");
 		try {
 			Files.write(testFileContent, f, Charsets.UTF_8);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/709fa1d9/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
index 39cb8ee..156098e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
@@ -21,7 +21,9 @@ package org.apache.flink.runtime.io.disk.iomanager;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.io.IOException;
@@ -34,12 +36,15 @@ import static org.junit.Assert.assertTrue;
 
 public class IOManagerTest {
 
+	@Rule
+	public final TemporaryFolder  temporaryFolder = new TemporaryFolder();
+
 	@Test
-	public void channelEnumerator() {
+	public void channelEnumerator() throws IOException {
 		IOManager ioMan = null;
 
 		try {
-			File tempPath = new File(System.getProperty("java.io.tmpdir"));
+			File tempPath = temporaryFolder.newFolder();
 
 			String[] tempDirs = new String[]{
 					new File(tempPath, "a").getAbsolutePath(),
@@ -126,4 +131,4 @@ public class IOManagerTest {
 			throw new UnsupportedOperationException();
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/709fa1d9/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
index 316fd21..544d473 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
@@ -20,14 +20,17 @@ package org.apache.flink.test.util;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
+
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Objects;
 
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -39,22 +42,23 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	
 	/** Configuration to start the testing cluster with */
 	protected final Configuration config;
-	
-	private final List<File> tempFiles;
-	
+
 	private final FiniteDuration timeout;
 
 	protected int taskManagerNumSlots = 1;
 
 	protected int numTaskManagers = 1;
-	
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+
 	/** The mini cluster that runs the test programs */
 	protected LocalFlinkMiniCluster executor;
 	
 
 	public AbstractTestBase(Configuration config) {
 		this.config = Objects.requireNonNull(config);
-		this.tempFiles = new ArrayList<File>();
 
 		timeout = AkkaUtils.getTimeout(config);
 	}
@@ -74,7 +78,6 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 
 	public void stopCluster() throws Exception {
 		stopCluster(executor, timeout);
-		deleteAllTempFiles();
 	}
 
 	//------------------
@@ -103,12 +106,12 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	// --------------------------------------------------------------------------------------------
 
 	public String getTempDirPath(String dirName) throws IOException {
-		File f = createAndRegisterTempFile(dirName);
+		File f = temporaryFolder.newFolder(dirName);
 		return f.toURI().toString();
 	}
 
 	public String getTempFilePath(String fileName) throws IOException {
-		File f = createAndRegisterTempFile(fileName);
+		File f = temporaryFolder.newFile(fileName);
 		return f.toURI().toString();
 	}
 
@@ -119,35 +122,7 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	}
 
 	public File createAndRegisterTempFile(String fileName) throws IOException {
-		File baseDir = new File(System.getProperty("java.io.tmpdir"));
-		File f = new File(baseDir, this.getClass().getName() + "-" + fileName);
-
-		if (f.exists()) {
-			deleteRecursively(f);
-		}
-
-		File parentToDelete = f;
-		while (true) {
-			File parent = parentToDelete.getParentFile();
-			if (parent == null) {
-				throw new IOException("Missed temp dir while traversing parents of a temp file.");
-			}
-			if (parent.equals(baseDir)) {
-				break;
-			}
-			parentToDelete = parent;
-		}
-
-		Files.createParentDirs(f);
-		this.tempFiles.add(parentToDelete);
-		return f;
+		return temporaryFolder.newFile(fileName);
 	}
 
-	private void deleteAllTempFiles() throws IOException {
-		for (File f : this.tempFiles) {
-			if (f.exists()) {
-				deleteRecursively(f);
-			}
-		}
-	}
 }


[09/19] flink git commit: [FLINK-5817] [test] (followup) Fix temporary folder and temp file path generation

Posted by se...@apache.org.
[FLINK-5817] [test] (followup) Fix temporary folder and temp file path generation

This makes sure the TemporaryFolder rule is already evaluated by the time
the temp files are generated.

This also injects a random parent directory to ensure that even for fix directory/file
names, the absolute path is randomized.


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

Branch: refs/heads/master
Commit: 1456f0a7084f45056ea9b09e3f85b1aae6b11c6e
Parents: 709fa1d
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Feb 19 16:21:07 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/test/util/AbstractTestBase.java   | 13 ++++++-------
 .../java/org/apache/flink/test/util/TestBaseUtils.java |  4 ++++
 2 files changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1456f0a7/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
index 544d473..be9b0b7 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
@@ -24,7 +24,7 @@ import com.google.common.io.Files;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
-import org.junit.Rule;
+import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 
 import scala.concurrent.duration.FiniteDuration;
@@ -49,8 +49,8 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 
 	protected int numTaskManagers = 1;
 
-	@Rule
-	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+	@ClassRule
+	public static final TemporaryFolder temporaryFolder = new TemporaryFolder();
 
 
 	/** The mini cluster that runs the test programs */
@@ -106,12 +106,12 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	// --------------------------------------------------------------------------------------------
 
 	public String getTempDirPath(String dirName) throws IOException {
-		File f = temporaryFolder.newFolder(dirName);
+		File f = createAndRegisterTempFile(dirName);
 		return f.toURI().toString();
 	}
 
 	public String getTempFilePath(String fileName) throws IOException {
-		File f = temporaryFolder.newFile(fileName);
+		File f = createAndRegisterTempFile(fileName);
 		return f.toURI().toString();
 	}
 
@@ -122,7 +122,6 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	}
 
 	public File createAndRegisterTempFile(String fileName) throws IOException {
-		return temporaryFolder.newFile(fileName);
+		return new File(temporaryFolder.newFolder(), fileName);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1456f0a7/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index 8431226..cc7c0e2 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -74,6 +74,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -283,6 +285,8 @@ public class TestBaseUtils extends TestLogger {
 			String resultPath,
 			String[] excludePrefixes,
 			boolean inOrderOfFiles) throws IOException {
+		
+		checkArgument(resultPath != null, "resultPath cannot be be null");
 
 		final BufferedReader[] readers = getResultReader(resultPath, excludePrefixes, inOrderOfFiles);
 		try {


[03/19] flink git commit: [FLINK-5497] [tests] Remove duplicated tests for hash tables

Posted by se...@apache.org.
[FLINK-5497] [tests] Remove duplicated tests for hash tables

This closes #3089


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

Branch: refs/heads/master
Commit: 53134594644407d0a3cd691b0e93ae09ff6c8102
Parents: 9f544d8
Author: Alexey Diomin <di...@gmail.com>
Authored: Tue Jan 10 22:04:41 2017 +0400
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 .../NonReusingReOpenableHashTableITCase.java    | 421 +-----------------
 .../hash/ReOpenableHashTableITCase.java         | 222 ++++++++++
 .../hash/ReOpenableHashTableTestBase.java       | 193 +++++++++
 .../hash/ReusingReOpenableHashTableITCase.java  | 429 +------------------
 4 files changed, 429 insertions(+), 836 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/53134594/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 576cbd4..6b4e170 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
@@ -19,190 +19,34 @@
 
 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.core.memory.MemoryType;
-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.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
 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;
 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.UnionIterator;
 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.io.IOException;
 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.FlatJoinFunction;
-import org.apache.flink.api.common.typeutils.GenericPairComparator;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
 
-import static org.junit.Assert.fail;
+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.
  */
-public class NonReusingReOpenableHashTableITCase {
+public class NonReusingReOpenableHashTableITCase extends ReOpenableHashTableTestBase {
 
-	private static final int PAGE_SIZE = 8 * 1024;
-	private static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages.
-
-	private static final long SEED1 = 561349061987311L;
-	private static final long SEED2 = 231434613412342L;
-
-	private static final int NUM_PROBES = 3; // number of reopenings of hash join
-
-	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 static final AbstractInvokable MEM_OWNER = new DummyInvokable();
-	private TypeSerializer<Tuple2<Integer, Integer>> recordBuildSideAccesssor;
-	private TypeSerializer<Tuple2<Integer, Integer>> recordProbeSideAccesssor;
-	private TypeComparator<Tuple2<Integer, Integer>> recordBuildSideComparator;
-	private TypeComparator<Tuple2<Integer, Integer>> recordProbeSideComparator;
-	private TypePairComparator<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> pactRecordComparator;
-
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	@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.recordBuildSideAccesssor = TestData.getIntIntTupleSerializer();
-		this.recordProbeSideAccesssor = TestData.getIntIntTupleSerializer();
-		this.recordBuildSideComparator = TestData.getIntIntTupleComparator();
-		this.recordProbeSideComparator = TestData.getIntIntTupleComparator();
-		this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator);
-
-		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
-		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 behavior with overflow buckets (Overflow buckets must be initialized correctly
-	 * if the input is reopened again)
-	 */
-	@Test
-	public void testOverflow() {
-
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TupleGenerator bgen = new TupleGenerator(SEED1, 200, 1024, KeyMode.RANDOM, ValueMode.FIX_LENGTH);
-			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-
-	/**
-	 * Verify proper operation if the build side is spilled to disk.
-	 */
-	@Test
-	public void testDoubleProbeSpilling() {
-
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TupleGenerator bgen = new TupleGenerator(SEED1, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-
-	/**
-	 * This test case verifies that hybrid hash join is able to handle multiple probe phases
-	 * when the build side fits completely into memory.
-	 */
-	@Test
-	public void testDoubleProbeInMemory() {
-
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TupleGenerator bgen = new TupleGenerator(SEED1, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-
-	private void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TupleGenerator bgen, TupleGenerator pgen) throws Exception {
+	protected void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TupleGenerator bgen, TupleGenerator pgen) throws Exception {
 		// collect expected data
-		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = NonReusingHashJoinIteratorITCase.joinTuples(NonReusingHashJoinIteratorITCase.collectTupleData(buildInput), NonReusingHashJoinIteratorITCase.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];
@@ -259,260 +103,5 @@ public class NonReusingReOpenableHashTableITCase {
 		iterator.close();
 	}
 
-	//
-	//
-	//	Tests taken from HahTableITCase!
-	//
-	//
-
-	private MutableObjectIterator<Tuple2<Integer, Integer>> getProbeInput(final int numKeys,
-			final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) {
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe1 = new UniformIntTupleGenerator(numKeys, probeValsPerKey, true);
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe2 = new TestData.ConstantIntIntTuplesIterator(repeatedValue1, 17, 5);
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe3 = new TestData.ConstantIntIntTuplesIterator(repeatedValue2, 23, 5);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> probes = new ArrayList<>();
-		probes.add(probe1);
-		probes.add(probe2);
-		probes.add(probe3);
-		return new UnionIterator<>(probes);
-	}
-
-	@Test
-	public void testSpillingHashJoinWithMassiveCollisions() throws IOException
-	{
-		// the following two values are known to have a hash-code collision on the initial level.
-		// we use them to make sure one partition grows over-proportionally large
-		final int REPEATED_VALUE_1 = 40559;
-		final int REPEATED_VALUE_2 = 92882;
-		final int REPEATED_VALUE_COUNT_BUILD = 200000;
-		final int REPEATED_VALUE_COUNT_PROBE = 5;
-
-		final int NUM_KEYS = 1000000;
-		final int BUILD_VALS_PER_KEY = 3;
-		final int PROBE_VALS_PER_KEY = 10;
-
-		// create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys
-		MutableObjectIterator<Tuple2<Integer, Integer>> build1 = new UniformIntTupleGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build2 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build3 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> builds = new ArrayList<>();
-		builds.add(build1);
-		builds.add(build2);
-		builds.add(build3);
-		MutableObjectIterator<Tuple2<Integer, Integer>> buildInput = new UnionIterator<>(builds);
-
-
-
-
-		// allocate the memory for the HashTable
-		List<MemorySegment> memSegments;
-		try {
-			memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896);
-		}
-		catch (MemoryAllocationException maex) {
-			fail("Memory for the Join could not be provided.");
-			return;
-		}
-
-		// create the map for validating the results
-		HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS);
-
-		// ----------------------------------------------------------------------------------------
-
-		final ReOpenableMutableHashTable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> join = new ReOpenableMutableHashTable<>(
-				this.recordBuildSideAccesssor, this.recordProbeSideAccesssor,
-				this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator,
-				memSegments, ioManager, true);
-
-		for (int probe = 0; probe < NUM_PROBES; probe++) {
-			// create a probe input that gives 10 million pairs with 10 values sharing a key
-			MutableObjectIterator<Tuple2<Integer, Integer>> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2);
-			if(probe == 0) {
-				join.open(buildInput, probeInput);
-			} else {
-				join.reopenProbe(probeInput);
-			}
-
-			Tuple2<Integer, Integer> record;
-			final Tuple2<Integer, Integer> recordReuse = new Tuple2<>();
-
-			while (join.nextRecord()) {
-				long numBuildValues = 0;
-
-				final Tuple2<Integer, Integer> probeRec = join.getCurrentProbeRecord();
-				Integer key = probeRec.f0;
-				
-				MutableObjectIterator<Tuple2<Integer, Integer>> buildSide = join.getBuildSideIterator();
-				if ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues = 1;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-				else {
-					fail("No build side values found for a probe key.");
-				}
-				while ((record = buildSide.next(record)) != null) {
-					numBuildValues++;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-
-				Long contained = map.get(key);
-				if (contained == null) {
-					contained = numBuildValues;
-				}
-				else {
-					contained = contained + numBuildValues;
-				}
-
-				map.put(key, contained);
-			}
-		}
-
-		join.close();
-
-		Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size());
-		for (Entry<Integer, Long> entry : map.entrySet()) {
-			long val = entry.getValue();
-			int key = entry.getKey();
-
-			if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
-							(PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val);
-			} else {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
-							PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val);
-			}
-		}
-
-
-		// ----------------------------------------------------------------------------------------
-
-		this.memoryManager.release(join.getFreedMemory());
-	}
-
-	/*
-	 * This test is basically identical to the "testSpillingHashJoinWithMassiveCollisions" test, only that the number
-	 * of repeated values (causing bucket collisions) are large enough to make sure that their target partition no longer
-	 * fits into memory by itself and needs to be repartitioned in the recursion again.
-	 */
-	@Test
-	public void testSpillingHashJoinWithTwoRecursions() throws IOException
-	{
-		// the following two values are known to have a hash-code collision on the first recursion level.
-		// we use them to make sure one partition grows over-proportionally large
-		final int REPEATED_VALUE_1 = 40559;
-		final int REPEATED_VALUE_2 = 92882;
-		final int REPEATED_VALUE_COUNT_BUILD = 200000;
-		final int REPEATED_VALUE_COUNT_PROBE = 5;
-
-		final int NUM_KEYS = 1000000;
-		final int BUILD_VALS_PER_KEY = 3;
-		final int PROBE_VALS_PER_KEY = 10;
-
-		// create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys
-		MutableObjectIterator<Tuple2<Integer, Integer>> build1 = new UniformIntTupleGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build2 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build3 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> builds = new ArrayList<>();
-		builds.add(build1);
-		builds.add(build2);
-		builds.add(build3);
-		MutableObjectIterator<Tuple2<Integer, Integer>> buildInput = new UnionIterator<>(builds);
-
-
-		// allocate the memory for the HashTable
-		List<MemorySegment> memSegments;
-		try {
-			memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896);
-		}
-		catch (MemoryAllocationException maex) {
-			fail("Memory for the Join could not be provided.");
-			return;
-		}
-
-		// create the map for validating the results
-		HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS);
-
-		// ----------------------------------------------------------------------------------------
-
-		final ReOpenableMutableHashTable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> join = new ReOpenableMutableHashTable<>(
-				this.recordBuildSideAccesssor, this.recordProbeSideAccesssor,
-				this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator,
-				memSegments, ioManager, true);
-		
-		for (int probe = 0; probe < NUM_PROBES; probe++) {
-			// create a probe input that gives 10 million pairs with 10 values sharing a key
-			MutableObjectIterator<Tuple2<Integer, Integer>> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2);
-			if (probe == 0) {
-				join.open(buildInput, probeInput);
-			} else {
-				join.reopenProbe(probeInput);
-			}
-			Tuple2<Integer, Integer> record;
-			final Tuple2<Integer, Integer> recordReuse = new Tuple2<>();
-
-			while (join.nextRecord()) {
-				long numBuildValues = 0;
-
-				final Tuple2<Integer, Integer> probeRec = join.getCurrentProbeRecord();
-				Integer key = probeRec.f0;
-				
-				MutableObjectIterator<Tuple2<Integer, Integer>> buildSide = join.getBuildSideIterator();
-				if ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues = 1;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-				else {
-					fail("No build side values found for a probe key.");
-				}
-				while ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues++;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-
-				Long contained = map.get(key);
-				if (contained == null) {
-					contained = numBuildValues;
-				}
-				else {
-					contained = contained + numBuildValues;
-				}
-
-				map.put(key, contained);
-			}
-		}
-
-		join.close();
-		Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size());
-		for (Entry<Integer, Long> entry : map.entrySet()) {
-			long val = entry.getValue();
-			int key = entry.getKey();
-
-			if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
-							(PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val);
-			} else {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
-							PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val);
-			}
-		}
-
-
-		// ----------------------------------------------------------------------------------------
-
-		this.memoryManager.release(join.getFreedMemory());
-	}
-
 
-	static Map<Integer, Collection<TupleMatch>> deepCopy(Map<Integer, Collection<TupleMatch>> expectedSecondMatchesMap) {
-		Map<Integer, Collection<TupleMatch>> copy = new HashMap<>(expectedSecondMatchesMap.size());
-		for(Entry<Integer, Collection<TupleMatch>> entry : expectedSecondMatchesMap.entrySet()) {
-			List<TupleMatch> matches = new ArrayList<TupleMatch>(entry.getValue().size());
-			for(TupleMatch m : entry.getValue()) {
-				matches.add(m);
-			}
-			copy.put(entry.getKey(), matches);
-		}
-		return copy;
-	}
-	
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/53134594/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java
new file mode 100644
index 0000000..f667c87
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java
@@ -0,0 +1,222 @@
+/*
+ * 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.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.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
+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.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.TestData;
+import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
+import org.apache.flink.runtime.operators.testutils.UnionIterator;
+import org.apache.flink.util.MutableObjectIterator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.fail;
+
+public class ReOpenableHashTableITCase {
+
+	private static final int PAGE_SIZE = 8 * 1024;
+	private static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages.
+
+	private static final int NUM_PROBES = 3; // number of reopenings of hash join
+
+	private IOManager ioManager;
+	private MemoryManager memoryManager;
+
+	private static final AbstractInvokable MEM_OWNER = new DummyInvokable();
+	private TypeSerializer<Tuple2<Integer, Integer>> recordBuildSideAccesssor;
+	private TypeSerializer<Tuple2<Integer, Integer>> recordProbeSideAccesssor;
+	private TypeComparator<Tuple2<Integer, Integer>> recordBuildSideComparator;
+	private TypeComparator<Tuple2<Integer, Integer>> recordProbeSideComparator;
+	private TypePairComparator<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> pactRecordComparator;
+
+	@SuppressWarnings({"unchecked", "rawtypes"})
+	@Before
+	public void beforeTest() {
+		this.recordBuildSideAccesssor = TestData.getIntIntTupleSerializer();
+		this.recordProbeSideAccesssor = TestData.getIntIntTupleSerializer();
+		this.recordBuildSideComparator = TestData.getIntIntTupleComparator();
+		this.recordProbeSideComparator = TestData.getIntIntTupleComparator();
+		this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator);
+
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
+		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;
+		}
+	}
+
+	private MutableObjectIterator<Tuple2<Integer, Integer>> getProbeInput(final int numKeys,
+																		  final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) {
+		MutableObjectIterator<Tuple2<Integer, Integer>> probe1 = new UniformIntTupleGenerator(numKeys, probeValsPerKey, true);
+		MutableObjectIterator<Tuple2<Integer, Integer>> probe2 = new TestData.ConstantIntIntTuplesIterator(repeatedValue1, 17, 5);
+		MutableObjectIterator<Tuple2<Integer, Integer>> probe3 = new TestData.ConstantIntIntTuplesIterator(repeatedValue2, 23, 5);
+		List<MutableObjectIterator<Tuple2<Integer, Integer>>> probes = new ArrayList<>();
+		probes.add(probe1);
+		probes.add(probe2);
+		probes.add(probe3);
+		return new UnionIterator<>(probes);
+	}
+
+	@Test
+	public void testSpillingHashJoinWithMassiveCollisions() throws IOException
+	{
+		// the following two values are known to have a hash-code collision on the initial level.
+		// we use them to make sure one partition grows over-proportionally large
+		final int REPEATED_VALUE_1 = 40559;
+		final int REPEATED_VALUE_2 = 92882;
+		final int REPEATED_VALUE_COUNT_BUILD = 200000;
+		final int REPEATED_VALUE_COUNT_PROBE = 5;
+
+		final int NUM_KEYS = 1000000;
+		final int BUILD_VALS_PER_KEY = 3;
+		final int PROBE_VALS_PER_KEY = 10;
+
+		// create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys
+		MutableObjectIterator<Tuple2<Integer, Integer>> build1 = new UniformIntTupleGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false);
+		MutableObjectIterator<Tuple2<Integer, Integer>> build2 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD);
+		MutableObjectIterator<Tuple2<Integer, Integer>> build3 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD);
+		List<MutableObjectIterator<Tuple2<Integer, Integer>>> builds = new ArrayList<>();
+		builds.add(build1);
+		builds.add(build2);
+		builds.add(build3);
+		MutableObjectIterator<Tuple2<Integer, Integer>> buildInput = new UnionIterator<>(builds);
+
+
+
+
+		// allocate the memory for the HashTable
+		List<MemorySegment> memSegments;
+		try {
+			memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896);
+		}
+		catch (MemoryAllocationException maex) {
+			fail("Memory for the Join could not be provided.");
+			return;
+		}
+
+		// create the map for validating the results
+		HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS);
+
+		// ----------------------------------------------------------------------------------------
+
+		final ReOpenableMutableHashTable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> join = new ReOpenableMutableHashTable<>(
+			this.recordBuildSideAccesssor, this.recordProbeSideAccesssor,
+			this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator,
+			memSegments, ioManager, true);
+
+		for (int probe = 0; probe < NUM_PROBES; probe++) {
+			// create a probe input that gives 10 million pairs with 10 values sharing a key
+			MutableObjectIterator<Tuple2<Integer, Integer>> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2);
+			if(probe == 0) {
+				join.open(buildInput, probeInput);
+			} else {
+				join.reopenProbe(probeInput);
+			}
+
+			Tuple2<Integer, Integer> record;
+			final Tuple2<Integer, Integer> recordReuse = new Tuple2<>();
+
+			while (join.nextRecord()) {
+				long numBuildValues = 0;
+
+				final Tuple2<Integer, Integer> probeRec = join.getCurrentProbeRecord();
+				Integer key = probeRec.f0;
+
+				MutableObjectIterator<Tuple2<Integer, Integer>> buildSide = join.getBuildSideIterator();
+				if ((record = buildSide.next(recordReuse)) != null) {
+					numBuildValues = 1;
+					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
+				}
+				else {
+					fail("No build side values found for a probe key.");
+				}
+				while ((record = buildSide.next(recordReuse)) != null) {
+					numBuildValues++;
+					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
+				}
+
+				Long contained = map.get(key);
+				if (contained == null) {
+					contained = numBuildValues;
+				}
+				else {
+					contained = contained + numBuildValues;
+				}
+
+				map.put(key, contained);
+			}
+		}
+
+		join.close();
+		Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size());
+		for (Map.Entry<Integer, Long> entry : map.entrySet()) {
+			long val = entry.getValue();
+			int key = entry.getKey();
+
+			if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) {
+				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
+					(PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val);
+			} else {
+				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key,
+					PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val);
+			}
+		}
+
+
+		// ----------------------------------------------------------------------------------------
+
+		this.memoryManager.release(join.getFreedMemory());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/53134594/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java
new file mode 100644
index 0000000..c1b87b0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java
@@ -0,0 +1,193 @@
+/*
+ * 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.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.core.memory.MemoryType;
+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.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.TupleGeneratorIterator;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+import java.util.*;
+
+public abstract class ReOpenableHashTableTestBase {
+
+	protected static final int PAGE_SIZE = 8 * 1024;
+	protected static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages.
+
+	protected static final long SEED1 = 561349061987311L;
+	protected static final long SEED2 = 231434613412342L;
+
+	protected static final int NUM_PROBES = 3; // number of reopenings of hash join
+
+	protected final AbstractInvokable parentTask = new DummyInvokable();
+
+	protected IOManager ioManager;
+	protected MemoryManager memoryManager;
+
+	protected TypeSerializer<Tuple2<Integer, String>> recordSerializer;
+	protected TypeComparator<Tuple2<Integer, String>> record1Comparator;
+	protected TypeComparator<Tuple2<Integer, String>> record2Comparator;
+	protected TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> recordPairComparator;
+
+	protected TypeSerializer<Tuple2<Integer, Integer>> recordBuildSideAccesssor;
+	protected TypeSerializer<Tuple2<Integer, Integer>> recordProbeSideAccesssor;
+	protected TypeComparator<Tuple2<Integer, Integer>> recordBuildSideComparator;
+	protected TypeComparator<Tuple2<Integer, Integer>> recordProbeSideComparator;
+	protected TypePairComparator<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> pactRecordComparator;
+
+	@SuppressWarnings({"unchecked", "rawtypes"})
+	@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.recordBuildSideAccesssor = TestData.getIntIntTupleSerializer();
+		this.recordProbeSideAccesssor = TestData.getIntIntTupleSerializer();
+		this.recordBuildSideComparator = TestData.getIntIntTupleComparator();
+		this.recordProbeSideComparator = TestData.getIntIntTupleComparator();
+		this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator);
+
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
+		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 behavior with overflow buckets (Overflow buckets must be initialized correctly
+	 * if the input is reopened again)
+	 */
+	@Test
+	public void testOverflow() {
+
+		int buildSize = 1000;
+		int probeSize = 1000;
+		try {
+			TupleGenerator bgen = new TupleGenerator(SEED1, 200, 1024, KeyMode.RANDOM, ValueMode.FIX_LENGTH);
+			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
+
+			final TupleGeneratorIterator buildInput = new TupleGeneratorIterator(bgen, buildSize);
+			final TupleGeneratorIterator probeInput = new TupleGeneratorIterator(pgen, probeSize);
+			doTest(buildInput,probeInput, bgen, pgen);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	/**
+	 * Verify proper operation if the build side is spilled to disk.
+	 */
+	@Test
+	public void testDoubleProbeSpilling() {
+
+		int buildSize = 1000;
+		int probeSize = 1000;
+		try {
+			TupleGenerator bgen = new TupleGenerator(SEED1, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
+			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
+
+			final TupleGeneratorIterator buildInput = new TupleGeneratorIterator(bgen, buildSize);
+			final TupleGeneratorIterator probeInput = new TupleGeneratorIterator(pgen, probeSize);
+			doTest(buildInput,probeInput, bgen, pgen);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	/**
+	 * This test case verifies that hybrid hash join is able to handle multiple probe phases
+	 * when the build side fits completely into memory.
+	 */
+	@Test
+	public void testDoubleProbeInMemory() {
+
+		int buildSize = 1000;
+		int probeSize = 1000;
+		try {
+			TupleGenerator bgen = new TupleGenerator(SEED1, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
+			TupleGenerator pgen = new TupleGenerator(SEED2, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
+
+			final TupleGeneratorIterator buildInput = new TupleGeneratorIterator(bgen, buildSize);
+			final TupleGeneratorIterator probeInput = new TupleGeneratorIterator(pgen, probeSize);
+
+			doTest(buildInput,probeInput, bgen, pgen);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("An exception occurred during the test: " + e.getMessage());
+		}
+	}
+
+	abstract protected void doTest(TupleGeneratorIterator buildInput, TupleGeneratorIterator probeInput, TestData.TupleGenerator bgen, TestData.TupleGenerator pgen) throws Exception;
+
+	static Map<Integer, Collection<TupleMatch>> deepCopy(Map<Integer, Collection<TupleMatch>> expectedSecondMatchesMap) {
+		Map<Integer, Collection<TupleMatch>> copy = new HashMap<>(expectedSecondMatchesMap.size());
+		for(Map.Entry<Integer, Collection<TupleMatch>> entry : expectedSecondMatchesMap.entrySet()) {
+			List<TupleMatch> matches = new ArrayList<TupleMatch>(entry.getValue().size());
+			for(TupleMatch m : entry.getValue()) {
+				matches.add(m);
+			}
+			copy.put(entry.getKey(), matches);
+		}
+		return copy;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/53134594/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 6afde16..af3a894 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
@@ -19,44 +19,21 @@
 
 package org.apache.flink.runtime.operators.hash;
 
-import static org.junit.Assert.fail;
+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.TestData;
+import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
 
-import java.io.IOException;
 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.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.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemoryType;
-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.MemoryAllocationException;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatchRemovingJoin;
-import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatch;
-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.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.UniformIntTupleGenerator;
-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.collectTupleData;
@@ -65,144 +42,9 @@ import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIterator
  * Test specialized hash join that keeps the build side data (in memory and on hard disk)
  * This is used for iterative tasks.
  */
-public class ReusingReOpenableHashTableITCase {
-	
-	private static final int PAGE_SIZE = 8 * 1024;
-	private static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages.
+public class ReusingReOpenableHashTableITCase extends ReOpenableHashTableTestBase {
 
-	private static final long SEED1 = 561349061987311L;
-	private static final long SEED2 = 231434613412342L;
-	
-	private static final int NUM_PROBES = 3; // number of reopenings of hash join
-	
-	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 static final AbstractInvokable MEM_OWNER = new DummyInvokable();
-	private TypeSerializer<Tuple2<Integer, Integer>> recordBuildSideAccesssor;
-	private TypeSerializer<Tuple2<Integer, Integer>> recordProbeSideAccesssor;
-	private TypeComparator<Tuple2<Integer, Integer>> recordBuildSideComparator;
-	private TypeComparator<Tuple2<Integer, Integer>> recordProbeSideComparator;
-	private TypePairComparator<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> pactRecordComparator;
-
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	@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.recordBuildSideAccesssor = TestData.getIntIntTupleSerializer();
-		this.recordProbeSideAccesssor = TestData.getIntIntTupleSerializer();
-		this.recordBuildSideComparator = TestData.getIntIntTupleComparator();
-		this.recordProbeSideComparator = TestData.getIntIntTupleComparator();
-		this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator);
-		
-		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
-		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 behavior with overflow buckets (Overflow buckets must be initialized correctly 
-	 * if the input is reopened again)
-	 */
-	@Test
-	public void testOverflow() {
-		
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TestData.TupleGenerator bgen = new TestData.TupleGenerator(SEED1, 200, 1024, KeyMode.RANDOM, ValueMode.FIX_LENGTH);
-			TestData.TupleGenerator pgen = new TestData.TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	/**
-	 * Verify proper operation if the build side is spilled to disk.
-	 */
-	@Test
-	public void testDoubleProbeSpilling() {
-		
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TestData.TupleGenerator bgen = new TestData.TupleGenerator(SEED1, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			TestData.TupleGenerator pgen = new TestData.TupleGenerator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	/**
-	 * This test case verifies that hybrid hash join is able to handle multiple probe phases
-	 * when the build side fits completely into memory.
-	 */
-	@Test
-	public void testDoubleProbeInMemory() {
-		
-		int buildSize = 1000;
-		int probeSize = 1000;
-		try {
-			TestData.TupleGenerator bgen = new TestData.TupleGenerator(SEED1, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			TestData.TupleGenerator pgen = new TestData.TupleGenerator(SEED2, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH);
-			
-			final TestData.TupleGeneratorIterator buildInput = new TestData.TupleGeneratorIterator(bgen, buildSize);
-			final TestData.TupleGeneratorIterator probeInput = new TestData.TupleGeneratorIterator(pgen, probeSize);
-			
-			doTest(buildInput,probeInput, bgen, pgen);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail("An exception occurred during the test: " + e.getMessage());
-		}
-	}
-	
-	private void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TestData.TupleGenerator bgen, TestData.TupleGenerator pgen) throws Exception {
+	protected void doTest(TestData.TupleGeneratorIterator buildInput, TestData.TupleGeneratorIterator probeInput, TupleGenerator bgen, TupleGenerator pgen) throws Exception {
 		// collect expected data
 		final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap = joinTuples(collectTupleData(buildInput), collectTupleData(probeInput));
 		
@@ -260,257 +102,4 @@ public class ReusingReOpenableHashTableITCase {
 		
 		iterator.close();
 	}
-	
-	//
-	//
-	//	Tests taken from HahTableITCase!
-	//
-	//
-	
-	private MutableObjectIterator<Tuple2<Integer, Integer>> getProbeInput(final int numKeys,
-			final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) {
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe1 = new UniformIntTupleGenerator(numKeys, probeValsPerKey, true);
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe2 = new TestData.ConstantIntIntTuplesIterator(repeatedValue1, 17, 5);
-		MutableObjectIterator<Tuple2<Integer, Integer>> probe3 = new TestData.ConstantIntIntTuplesIterator(repeatedValue2, 23, 5);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> probes = new ArrayList<>();
-		probes.add(probe1);
-		probes.add(probe2);
-		probes.add(probe3);
-		return new UnionIterator<>(probes);
-	}
-	
-	@Test
-	public void testSpillingHashJoinWithMassiveCollisions() throws IOException {
-		// the following two values are known to have a hash-code collision on the initial level.
-		// we use them to make sure one partition grows over-proportionally large
-		final int REPEATED_VALUE_1 = 40559;
-		final int REPEATED_VALUE_2 = 92882;
-		final int REPEATED_VALUE_COUNT_BUILD = 200000;
-		final int REPEATED_VALUE_COUNT_PROBE = 5;
-		
-		final int NUM_KEYS = 1000000;
-		final int BUILD_VALS_PER_KEY = 3;
-		final int PROBE_VALS_PER_KEY = 10;
-		
-		// create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys
-		MutableObjectIterator<Tuple2<Integer, Integer>> build1 = new UniformIntTupleGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build2 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build3 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> builds = new ArrayList<>();
-		builds.add(build1);
-		builds.add(build2);
-		builds.add(build3);
-		MutableObjectIterator<Tuple2<Integer, Integer>> buildInput = new UnionIterator<>(builds);
-
-		// allocate the memory for the HashTable
-		List<MemorySegment> memSegments;
-		try {
-			memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896);
-		}
-		catch (MemoryAllocationException maex) {
-			fail("Memory for the Join could not be provided.");
-			return;
-		}
-		
-		// create the map for validating the results
-		HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS);
-		
-		// ----------------------------------------------------------------------------------------
-		
-		final ReOpenableMutableHashTable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> join = new ReOpenableMutableHashTable<>(
-				this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, 
-				this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator,
-				memSegments, ioManager, true);
-		
-		for(int probe = 0; probe < NUM_PROBES; probe++) {
-			// create a probe input that gives 10 million pairs with 10 values sharing a key
-			MutableObjectIterator<Tuple2<Integer, Integer>> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2);
-			if(probe == 0) {
-				join.open(buildInput, probeInput);
-			} else {
-				join.reopenProbe(probeInput);
-			}
-		
-			Tuple2<Integer, Integer> record;
-			final Tuple2<Integer, Integer> recordReuse = new Tuple2<>();
-
-			while (join.nextRecord()) {
-				long numBuildValues = 0;
-		
-				final Tuple2<Integer, Integer> probeRec = join.getCurrentProbeRecord();
-				Integer key = probeRec.f0;
-				
-				MutableObjectIterator<Tuple2<Integer, Integer>> buildSide = join.getBuildSideIterator();
-				if ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues = 1;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0); 
-				}
-				else {
-					fail("No build side values found for a probe key.");
-				}
-				while ((record = buildSide.next(record)) != null) {
-					numBuildValues++;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-				
-				Long contained = map.get(key);
-				if (contained == null) {
-					contained = numBuildValues;
-				}
-				else {
-					contained = contained + numBuildValues;
-				}
-				
-				map.put(key, contained);
-			}
-		}
-		
-		join.close();
-		
-		Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size());
-		for (Map.Entry<Integer, Long> entry : map.entrySet()) {
-			long val = entry.getValue();
-			int key = entry.getKey();
-	
-			if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, 
-							(PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val);
-			} else {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, 
-							PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val);
-			}
-		}
-		
-		
-		// ----------------------------------------------------------------------------------------
-		
-		this.memoryManager.release(join.getFreedMemory());
-	}
-	
-	/*
-	 * This test is basically identical to the "testSpillingHashJoinWithMassiveCollisions" test, only that the number
-	 * of repeated values (causing bucket collisions) are large enough to make sure that their target partition no longer
-	 * fits into memory by itself and needs to be repartitioned in the recursion again.
-	 */
-	@Test
-	public void testSpillingHashJoinWithTwoRecursions() throws IOException
-	{
-		// the following two values are known to have a hash-code collision on the first recursion level.
-		// we use them to make sure one partition grows over-proportionally large
-		final int REPEATED_VALUE_1 = 40559;
-		final int REPEATED_VALUE_2 = 92882;
-		final int REPEATED_VALUE_COUNT_BUILD = 200000;
-		final int REPEATED_VALUE_COUNT_PROBE = 5;
-		
-		final int NUM_KEYS = 1000000;
-		final int BUILD_VALS_PER_KEY = 3;
-		final int PROBE_VALS_PER_KEY = 10;
-		
-		// create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys
-		MutableObjectIterator<Tuple2<Integer, Integer>> build1 = new UniformIntTupleGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build2 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD);
-		MutableObjectIterator<Tuple2<Integer, Integer>> build3 = new TestData.ConstantIntIntTuplesIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD);
-		List<MutableObjectIterator<Tuple2<Integer, Integer>>> builds = new ArrayList<>();
-		builds.add(build1);
-		builds.add(build2);
-		builds.add(build3);
-		MutableObjectIterator<Tuple2<Integer, Integer>> buildInput = new UnionIterator<>(builds);
-	
-
-		// allocate the memory for the HashTable
-		List<MemorySegment> memSegments;
-		try {
-			memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896);
-		}
-		catch (MemoryAllocationException maex) {
-			fail("Memory for the Join could not be provided.");
-			return;
-		}
-		
-		// create the map for validating the results
-		HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS);
-		
-		// ----------------------------------------------------------------------------------------
-		
-		final ReOpenableMutableHashTable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> join = new ReOpenableMutableHashTable<>(
-				this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, 
-				this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator,
-				memSegments, ioManager, true);
-		
-		for (int probe = 0; probe < NUM_PROBES; probe++) {
-			// create a probe input that gives 10 million pairs with 10 values sharing a key
-			MutableObjectIterator<Tuple2<Integer, Integer>> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2);
-			if(probe == 0) {
-				join.open(buildInput, probeInput);
-			} else {
-				join.reopenProbe(probeInput);
-			}
-			Tuple2<Integer, Integer> record;
-			final Tuple2<Integer, Integer> recordReuse = new Tuple2<>();
-
-			while (join.nextRecord())
-			{	
-				long numBuildValues = 0;
-				
-				final Tuple2<Integer, Integer> probeRec = join.getCurrentProbeRecord();
-				Integer key = probeRec.f0;
-				
-				MutableObjectIterator<Tuple2<Integer, Integer>> buildSide = join.getBuildSideIterator();
-				if ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues = 1;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0); 
-				}
-				else {
-					fail("No build side values found for a probe key.");
-				}
-				while ((record = buildSide.next(recordReuse)) != null) {
-					numBuildValues++;
-					Assert.assertEquals("Probe-side key was different than build-side key.", key, record.f0);
-				}
-				
-				Long contained = map.get(key);
-				if (contained == null) {
-					contained = numBuildValues;
-				}
-				else {
-					contained = contained + numBuildValues;
-				}
-				
-				map.put(key, contained);
-			}
-		}
-		
-		join.close();
-		Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size());
-		for (Map.Entry<Integer, Long> entry : map.entrySet()) {
-			long val = entry.getValue();
-			int key = entry.getKey();
-	
-			if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, 
-							(PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val);
-			} else {
-				Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, 
-							PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val);
-			}
-		}
-		
-		
-		// ----------------------------------------------------------------------------------------
-		
-		this.memoryManager.release(join.getFreedMemory());
-	}
-	
-	
-	static Map<Integer, Collection<TupleMatch>> deepCopy(Map<Integer, Collection<TupleMatch>> expectedSecondMatchesMap) {
-		Map<Integer, Collection<TupleMatch>> copy = new HashMap<>(expectedSecondMatchesMap.size());
-		for(Map.Entry<Integer, Collection<TupleMatch>> entry : expectedSecondMatchesMap.entrySet()) {
-			List<TupleMatch> matches = new ArrayList<TupleMatch>(entry.getValue().size());
-			for(TupleMatch m : entry.getValue()) {
-				matches.add(m);
-			}
-			copy.put(entry.getKey(), matches);
-		}
-		return copy;
-	}
 }


[02/19] flink git commit: [FLINK-5522] [storm compatibility] Move Storm LocalCluster based test to a separate class

Posted by se...@apache.org.
[FLINK-5522] [storm compatibility] Move Storm LocalCluster based test to a separate class

This fixes the problem that the Storm LocalCluster can't run with powermock

This closes #3138


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

Branch: refs/heads/master
Commit: d05fc377ee688b231fb1b0daeb8a34fd054f3ca1
Parents: 5313459
Author: liuyuzhong7 <li...@gmail.com>
Authored: Thu Feb 9 16:16:15 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 .../storm/wrappers/WrapperSetupHelperTest.java  | 167 +---------------
 .../WrapperSetupInLocalClusterTest.java         | 190 +++++++++++++++++++
 2 files changed, 191 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d05fc377/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
index 5e29ac4..5f38705 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
@@ -17,29 +17,15 @@
 
 package org.apache.flink.storm.wrappers;
 
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.generated.ComponentCommon;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.task.TopologyContext;
+import org.apache.flink.storm.util.AbstractTest;
 import org.apache.storm.topology.IComponent;
 import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.topology.IRichSpout;
-import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.utils.Utils;
-
-import org.apache.flink.storm.api.FlinkTopology;
-import org.apache.flink.storm.util.AbstractTest;
-import org.apache.flink.storm.util.TestDummyBolt;
-import org.apache.flink.storm.util.TestDummySpout;
-import org.apache.flink.storm.util.TestSink;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -47,14 +33,9 @@ import org.powermock.modules.junit4.PowerMockRunner;
 
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
 
 import static java.util.Collections.singleton;
-
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(WrapperSetupHelper.class)
@@ -150,150 +131,4 @@ public class WrapperSetupHelperTest extends AbstractTest {
 				boltOrSpout,
 				numberOfAttributes == -1 ? new HashSet<String>(singleton(Utils.DEFAULT_STREAM_ID)) : null));
 	}
-
-	@Test
-	public void testCreateTopologyContext() {
-		HashMap<String, Integer> dops = new HashMap<String, Integer>();
-		dops.put("spout1", 1);
-		dops.put("spout2", 3);
-		dops.put("bolt1", 1);
-		dops.put("bolt2", 2);
-		dops.put("sink", 1);
-
-		HashMap<String, Integer> taskCounter = new HashMap<String, Integer>();
-		taskCounter.put("spout1", 0);
-		taskCounter.put("spout2", 0);
-		taskCounter.put("bolt1", 0);
-		taskCounter.put("bolt2", 0);
-		taskCounter.put("sink", 0);
-
-		HashMap<String, IComponent> operators = new HashMap<String, IComponent>();
-		operators.put("spout1", new TestDummySpout());
-		operators.put("spout2", new TestDummySpout());
-		operators.put("bolt1", new TestDummyBolt());
-		operators.put("bolt2", new TestDummyBolt());
-		operators.put("sink", new TestSink());
-
-		TopologyBuilder builder = new TopologyBuilder();
-
-		builder.setSpout("spout1", (IRichSpout) operators.get("spout1"), dops.get("spout1"));
-		builder.setSpout("spout2", (IRichSpout) operators.get("spout2"), dops.get("spout2"));
-		builder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1");
-		builder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2");
-		builder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink"))
-				.shuffleGrouping("bolt1", TestDummyBolt.groupingStreamId)
-				.shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId)
-				.shuffleGrouping("bolt2", TestDummyBolt.groupingStreamId)
-				.shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId);
-
-//		LocalCluster cluster = new LocalCluster();
-//		Config c = new Config();
-//		c.setNumAckers(0);
-//		cluster.submitTopology("test", c, builder.createTopology());
-//
-//		while (TestSink.result.size() != 8) {
-//			Utils.sleep(100);
-//		}
-//		cluster.shutdown();
-
-		final FlinkTopology flinkBuilder = FlinkTopology.createTopology(builder);
-		StormTopology stormTopology = flinkBuilder.getStormTopology();
-
-		Set<Integer> taskIds = new HashSet<Integer>();
-
-		for (TopologyContext expectedContext : TestSink.result) {
-			final String thisComponentId = expectedContext.getThisComponentId();
-			int index = taskCounter.get(thisComponentId);
-
-			StreamingRuntimeContext context = mock(StreamingRuntimeContext.class);
-			when(context.getTaskName()).thenReturn(thisComponentId);
-			when(context.getNumberOfParallelSubtasks()).thenReturn(dops.get(thisComponentId));
-			when(context.getIndexOfThisSubtask()).thenReturn(index);
-			taskCounter.put(thisComponentId, ++index);
-
-			Config stormConfig = new Config();
-			stormConfig.put(WrapperSetupHelper.TOPOLOGY_NAME, "test");
-
-			TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext(context,
-					operators.get(thisComponentId), thisComponentId, stormTopology, stormConfig);
-
-			ComponentCommon expcetedCommon = expectedContext.getComponentCommon(thisComponentId);
-			ComponentCommon common = topologyContext.getComponentCommon(thisComponentId);
-
-			Assert.assertNull(topologyContext.getCodeDir());
-			Assert.assertNull(common.get_json_conf());
-			Assert.assertNull(topologyContext.getExecutorData(null));
-			Assert.assertNull(topologyContext.getPIDDir());
-			Assert.assertNull(topologyContext.getResource(null));
-			Assert.assertNull(topologyContext.getSharedExecutor());
-			Assert.assertNull(expectedContext.getTaskData(null));
-			Assert.assertNull(topologyContext.getThisWorkerPort());
-
-			Assert.assertTrue(expectedContext.getStormId().startsWith(topologyContext.getStormId()));
-
-			Assert.assertEquals(expcetedCommon.get_inputs(), common.get_inputs());
-			Assert.assertEquals(expcetedCommon.get_parallelism_hint(), common.get_parallelism_hint());
-			Assert.assertEquals(expcetedCommon.get_streams(), common.get_streams());
-			Assert.assertEquals(expectedContext.getComponentIds(), topologyContext.getComponentIds());
-			Assert.assertEquals(expectedContext.getComponentStreams(thisComponentId),
-					topologyContext.getComponentStreams(thisComponentId));
-			Assert.assertEquals(thisComponentId, topologyContext.getThisComponentId());
-			Assert.assertEquals(expectedContext.getThisSources(), topologyContext.getThisSources());
-			Assert.assertEquals(expectedContext.getThisStreams(), topologyContext.getThisStreams());
-			Assert.assertEquals(expectedContext.getThisTargets(), topologyContext.getThisTargets());
-			Assert.assertEquals(0, topologyContext.getThisWorkerTasks().size());
-
-			for (int taskId : topologyContext.getComponentTasks(thisComponentId)) {
-				Assert.assertEquals(thisComponentId, topologyContext.getComponentId(taskId));
-			}
-
-			for (String componentId : expectedContext.getComponentIds()) {
-				Assert.assertEquals(expectedContext.getSources(componentId),
-						topologyContext.getSources(componentId));
-				Assert.assertEquals(expectedContext.getTargets(componentId),
-						topologyContext.getTargets(componentId));
-
-				for (String streamId : expectedContext.getComponentStreams(componentId)) {
-					Assert.assertEquals(
-							expectedContext.getComponentOutputFields(componentId, streamId).toList(),
-							topologyContext.getComponentOutputFields(componentId, streamId).toList());
-				}
-			}
-
-			for (String streamId : expectedContext.getThisStreams()) {
-				Assert.assertEquals(expectedContext.getThisOutputFields(streamId).toList(),
-						topologyContext.getThisOutputFields(streamId).toList());
-			}
-
-			HashMap<Integer, String> taskToComponents = new HashMap<Integer, String>();
-			Set<Integer> allTaskIds = new HashSet<Integer>();
-			for (String componentId : expectedContext.getComponentIds()) {
-				List<Integer> possibleTasks = expectedContext.getComponentTasks(componentId);
-				List<Integer> tasks = topologyContext.getComponentTasks(componentId);
-
-				Iterator<Integer> p_it = possibleTasks.iterator();
-				Iterator<Integer> t_it = tasks.iterator();
-				while(p_it.hasNext()) {
-					Assert.assertTrue(t_it.hasNext());
-					Assert.assertNull(taskToComponents.put(p_it.next(), componentId));
-					Assert.assertTrue(allTaskIds.add(t_it.next()));
-				}
-				Assert.assertFalse(t_it.hasNext());
-			}
-
-			Assert.assertEquals(taskToComponents, expectedContext.getTaskToComponent());
-			Assert.assertTrue(taskIds.add(topologyContext.getThisTaskId()));
-
-			try {
-				topologyContext.getHooks();
-				Assert.fail();
-			} catch (UnsupportedOperationException e) { /* expected */ }
-
-			try {
-				topologyContext.getRegisteredMetricByName(null);
-				Assert.fail();
-			} catch (UnsupportedOperationException e) { /* expected */ }
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d05fc377/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupInLocalClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupInLocalClusterTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupInLocalClusterTest.java
new file mode 100644
index 0000000..00173df
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupInLocalClusterTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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.storm.wrappers;
+
+import org.apache.flink.storm.api.FlinkTopology;
+import org.apache.flink.storm.util.AbstractTest;
+import org.apache.flink.storm.util.TestDummyBolt;
+import org.apache.flink.storm.util.TestDummySpout;
+import org.apache.flink.storm.util.TestSink;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.generated.ComponentCommon;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IComponent;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.utils.Utils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.*;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class WrapperSetupInLocalClusterTest extends AbstractTest {
+
+	@Test
+	public void testCreateTopologyContext() {
+		HashMap<String, Integer> dops = new HashMap<String, Integer>();
+		dops.put("spout1", 1);
+		dops.put("spout2", 3);
+		dops.put("bolt1", 1);
+		dops.put("bolt2", 2);
+		dops.put("sink", 1);
+
+		HashMap<String, Integer> taskCounter = new HashMap<String, Integer>();
+		taskCounter.put("spout1", 0);
+		taskCounter.put("spout2", 0);
+		taskCounter.put("bolt1", 0);
+		taskCounter.put("bolt2", 0);
+		taskCounter.put("sink", 0);
+
+		HashMap<String, IComponent> operators = new HashMap<String, IComponent>();
+		operators.put("spout1", new TestDummySpout());
+		operators.put("spout2", new TestDummySpout());
+		operators.put("bolt1", new TestDummyBolt());
+		operators.put("bolt2", new TestDummyBolt());
+		operators.put("sink", new TestSink());
+
+		TopologyBuilder builder = new TopologyBuilder();
+
+		builder.setSpout("spout1", (IRichSpout) operators.get("spout1"), dops.get("spout1"));
+		builder.setSpout("spout2", (IRichSpout) operators.get("spout2"), dops.get("spout2"));
+		builder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1");
+		builder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2");
+		builder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink"))
+				.shuffleGrouping("bolt1", TestDummyBolt.groupingStreamId)
+				.shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId)
+				.shuffleGrouping("bolt2", TestDummyBolt.groupingStreamId)
+				.shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId);
+
+		LocalCluster cluster = new LocalCluster();
+		Config c = new Config();
+		c.setNumAckers(0);
+		cluster.submitTopology("test", c, builder.createTopology());
+
+		while (TestSink.result.size() != 8) {
+			Utils.sleep(100);
+		}
+		cluster.shutdown();
+		final FlinkTopology flinkBuilder = FlinkTopology.createTopology(builder);
+		StormTopology stormTopology = flinkBuilder.getStormTopology();
+
+		Set<Integer> taskIds = new HashSet<Integer>();
+
+		for (TopologyContext expectedContext : TestSink.result) {
+			final String thisComponentId = expectedContext.getThisComponentId();
+			int index = taskCounter.get(thisComponentId);
+
+			StreamingRuntimeContext context = mock(StreamingRuntimeContext.class);
+			when(context.getTaskName()).thenReturn(thisComponentId);
+			when(context.getNumberOfParallelSubtasks()).thenReturn(dops.get(thisComponentId));
+			when(context.getIndexOfThisSubtask()).thenReturn(index);
+			taskCounter.put(thisComponentId, ++index);
+
+			Config stormConfig = new Config();
+			stormConfig.put(WrapperSetupHelper.TOPOLOGY_NAME, "test");
+
+			TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext(context,
+				operators.get(thisComponentId), thisComponentId, stormTopology, stormConfig);
+
+			ComponentCommon expcetedCommon = expectedContext.getComponentCommon(thisComponentId);
+			ComponentCommon common = topologyContext.getComponentCommon(thisComponentId);
+
+			Assert.assertNull(topologyContext.getCodeDir());
+			Assert.assertNull(common.get_json_conf());
+			Assert.assertNull(topologyContext.getExecutorData(null));
+			Assert.assertNull(topologyContext.getPIDDir());
+			Assert.assertNull(topologyContext.getResource(null));
+			Assert.assertNull(topologyContext.getSharedExecutor());
+			Assert.assertNull(expectedContext.getTaskData(null));
+			Assert.assertNull(topologyContext.getThisWorkerPort());
+
+			Assert.assertTrue(expectedContext.getStormId().startsWith(topologyContext.getStormId()));
+
+			Assert.assertEquals(expcetedCommon.get_inputs(), common.get_inputs());
+			Assert.assertEquals(expcetedCommon.get_parallelism_hint(), common.get_parallelism_hint());
+			Assert.assertEquals(expcetedCommon.get_streams(), common.get_streams());
+			Assert.assertEquals(expectedContext.getComponentIds(), topologyContext.getComponentIds());
+			Assert.assertEquals(expectedContext.getComponentStreams(thisComponentId),
+				topologyContext.getComponentStreams(thisComponentId));
+			Assert.assertEquals(thisComponentId, topologyContext.getThisComponentId());
+			Assert.assertEquals(expectedContext.getThisSources(), topologyContext.getThisSources());
+			Assert.assertEquals(expectedContext.getThisStreams(), topologyContext.getThisStreams());
+			Assert.assertEquals(expectedContext.getThisTargets(), topologyContext.getThisTargets());
+			Assert.assertEquals(0, topologyContext.getThisWorkerTasks().size());
+
+			for (int taskId : topologyContext.getComponentTasks(thisComponentId)) {
+				Assert.assertEquals(thisComponentId, topologyContext.getComponentId(taskId));
+			}
+
+			for (String componentId : expectedContext.getComponentIds()) {
+				Assert.assertEquals(expectedContext.getSources(componentId),
+					topologyContext.getSources(componentId));
+				Assert.assertEquals(expectedContext.getTargets(componentId),
+					topologyContext.getTargets(componentId));
+
+				for (String streamId : expectedContext.getComponentStreams(componentId)) {
+					Assert.assertEquals(
+						expectedContext.getComponentOutputFields(componentId, streamId).toList(),
+						topologyContext.getComponentOutputFields(componentId, streamId).toList());
+				}
+			}
+
+			for (String streamId : expectedContext.getThisStreams()) {
+				Assert.assertEquals(expectedContext.getThisOutputFields(streamId).toList(),
+					topologyContext.getThisOutputFields(streamId).toList());
+			}
+
+			HashMap<Integer, String> taskToComponents = new HashMap<Integer, String>();
+			Set<Integer> allTaskIds = new HashSet<Integer>();
+			for (String componentId : expectedContext.getComponentIds()) {
+				List<Integer> possibleTasks = expectedContext.getComponentTasks(componentId);
+				List<Integer> tasks = topologyContext.getComponentTasks(componentId);
+
+				Iterator<Integer> p_it = possibleTasks.iterator();
+				Iterator<Integer> t_it = tasks.iterator();
+				while(p_it.hasNext()) {
+					Assert.assertTrue(t_it.hasNext());
+					Assert.assertNull(taskToComponents.put(p_it.next(), componentId));
+					Assert.assertTrue(allTaskIds.add(t_it.next()));
+				}
+				Assert.assertFalse(t_it.hasNext());
+			}
+
+			Assert.assertEquals(taskToComponents, expectedContext.getTaskToComponent());
+			Assert.assertTrue(taskIds.add(topologyContext.getThisTaskId()));
+
+			try {
+				topologyContext.getHooks();
+				Assert.fail();
+			} catch (UnsupportedOperationException e) { /* expected */ }
+
+			try {
+				topologyContext.getRegisteredMetricByName(null);
+				Assert.fail();
+			} catch (UnsupportedOperationException e) { /* expected */ }
+		}
+	}
+
+}


[10/19] flink git commit: [FLINK-4813] [test-utils] Make the hadoop-minikdc dependency optional

Posted by se...@apache.org.
[FLINK-4813] [test-utils] Make the hadoop-minikdc dependency optional

With this change, any project using flink-test-utils which also requires
SecureTestEnvironment must add a dependency to hadoop-minikdc itself, e.g. in
pom.xml:

   ...
   <dependencies>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minikdc</artifactId>
       <version>${minikdc.version}</version>
       <scope>compile</scope>
     </dependency>
   ...
   </dependencies>
   ...

   <build>
     <plugins>
       <!--
         https://issues.apache.org/jira/browse/DIRSHARED-134
         Required to pull the Mini-KDC transitive dependency
       -->
       <plugin>
       <groupId>org.apache.felix</groupId>
       <artifactId>maven-bundle-plugin</artifactId>
       <version>3.0.1</version>
       <inherited>true</inherited>
       <extensions>true</extensions>
     </plugin>
   ...

This closes #3322


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

Branch: refs/heads/master
Commit: 391efd35bffe3f5796cd655ae1598f102a2e8fb7
Parents: 1ceb7d8
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Wed Feb 15 14:24:32 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../flink-test-utils/pom.xml                    | 23 +++++++++++++
 .../flink/test/util/SecureTestEnvironment.java  | 34 ++++++++++++++++++--
 pom.xml                                         | 10 ------
 3 files changed, 55 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/391efd35/flink-test-utils-parent/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml
index b986056..069f1da 100644
--- a/flink-test-utils-parent/flink-test-utils/pom.xml
+++ b/flink-test-utils-parent/flink-test-utils/pom.xml
@@ -83,8 +83,31 @@ under the License.
 			<artifactId>hadoop-minikdc</artifactId>
 			<version>${minikdc.version}</version>
 			<scope>compile</scope>
+			<!-- Since this requires the maven-bundle-plugin to be present,
+			make it optional.
+			The only class using this dependency is SecureTestEnvironment and if
+			a project is using it, it must now add the hadoop-minikdc dependency
+			on its own as well as include maven-bundle-plugin.
+			-->
+			<optional>true</optional>
 		</dependency>
 
 	</dependencies>
 
+	<build>
+		<plugins>
+			<!--
+            https://issues.apache.org/jira/browse/DIRSHARED-134
+            Required to pull the Mini-KDC transitive dependency
+            -->
+			<plugin>
+				<groupId>org.apache.felix</groupId>
+				<artifactId>maven-bundle-plugin</artifactId>
+				<version>3.0.1</version>
+				<inherited>true</inherited>
+				<extensions>true</extensions>
+			</plugin>
+		</plugins>
+	</build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/391efd35/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
index 10450c3..febd074 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
@@ -37,9 +37,39 @@ import java.util.Properties;
 /**
  * Helper {@link SecureTestEnvironment} to handle MiniKDC lifecycle.
  * This class can be used to start/stop MiniKDC and create secure configurations for MiniDFSCluster
- * and MiniYarn
+ * and MiniYarn.
+ *
+ * If you use this class in your project, please make sure to add a dependency to
+ * <tt>hadoop-minikdc</tt>, e.g. in your <tt>pom.xml</tt>:
+ * <pre>{@code
+ * ...
+ * <dependencies>
+ *   <dependency>
+ *     <groupId>org.apache.hadoop</groupId>
+ *     <artifactId>hadoop-minikdc</artifactId>
+ *     <version>${minikdc.version}</version>
+ *     <scope>compile</scope>
+ *   </dependency>
+ * ...
+ * </dependencies>
+ * ...
+ *
+ * <build>
+ *   <plugins>
+ *     <!--
+ *       https://issues.apache.org/jira/browse/DIRSHARED-134
+ *       Required to pull the Mini-KDC transitive dependency
+ *     -->
+ *     <plugin>
+ *     <groupId>org.apache.felix</groupId>
+ *     <artifactId>maven-bundle-plugin</artifactId>
+ *     <version>3.0.1</version>
+ *     <inherited>true</inherited>
+ *     <extensions>true</extensions>
+ *   </plugin>
+ * ...
+ * }</pre>
  */
-
 public class SecureTestEnvironment {
 
 	protected static final Logger LOG = LoggerFactory.getLogger(SecureTestEnvironment.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/391efd35/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e6fae81..d81296e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1120,16 +1120,6 @@ under the License.
 				</executions>
 			</plugin>
 
-			<!-- Pull bundled transitive dependencies (i.e. Mini-KDC).
-			See https://issues.apache.org/jira/browse/DIRSHARED-134 -->
-			<plugin>
-				<groupId>org.apache.felix</groupId>
-				<artifactId>maven-bundle-plugin</artifactId>
-				<version>3.0.1</version>
-				<inherited>true</inherited>
-				<extensions>true</extensions>
-			</plugin>
-
 		</plugins>
 
 		<!-- Plugin configurations for plugins activated in sub-projects --> 


[04/19] flink git commit: [FLINK-5129] [distributed runtime] BlobCache to directly accesses Blobs from distrinbuted file system if possible

Posted by se...@apache.org.
[FLINK-5129] [distributed runtime] BlobCache to directly accesses Blobs from distrinbuted file system if possible

This closes #3084


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

Branch: refs/heads/master
Commit: 9f544d83b3443cf33f5890efdb956678847d445f
Parents: e68ee5c
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Tue Nov 22 12:49:03 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 .../handlers/TaskManagerLogHandler.java         |   2 +-
 .../apache/flink/runtime/blob/BlobCache.java    | 269 +++++++++++--------
 .../apache/flink/runtime/blob/BlobClient.java   |   3 +-
 .../apache/flink/runtime/blob/BlobServer.java   |  56 +---
 .../runtime/blob/BlobServerConnection.java      |   8 +
 .../apache/flink/runtime/blob/BlobStore.java    |  29 +-
 .../apache/flink/runtime/blob/BlobUtils.java    |  75 +++---
 .../flink/runtime/blob/FileSystemBlobStore.java |  34 +--
 .../flink/runtime/blob/VoidBlobStore.java       |   9 +-
 .../apache/flink/runtime/client/JobClient.java  |   8 +-
 .../librarycache/BlobLibraryCacheManager.java   |  13 +-
 .../highavailability/ZookeeperHaServices.java   |  20 +-
 .../runtime/taskexecutor/TaskExecutor.java      |  19 +-
 .../runtime/blob/BlobCacheRetriesTest.java      |  86 +++++-
 .../runtime/blob/BlobCacheSuccessTest.java      |  76 +++++-
 .../flink/runtime/blob/BlobRecoveryITCase.java  |  31 +--
 .../runtime/blob/BlobServerDeleteTest.java      |  66 ++---
 .../flink/runtime/blob/BlobServerRangeTest.java |   1 +
 .../flink/runtime/blob/BlobUtilsTest.java       |   6 +-
 .../BlobLibraryCacheRecoveryITCase.java         |  18 +-
 20 files changed, 498 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
index 78c4455..6583d3b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
@@ -150,7 +150,7 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
 			scala.concurrent.Future<Object> portFuture = jobManager.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout);
 			scala.concurrent.Future<BlobCache> cacheFuture = portFuture.map(new Mapper<Object, BlobCache>() {
 				@Override
-				public BlobCache apply(Object result) {
+				public BlobCache checkedApply(Object result) throws IOException {
 					Option<String> hostOption = jobManager.actor().path().address().host();
 					String host = hostOption.isDefined() ? hostOption.get() : "localhost";
 					int port = (int) result;

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
index 7ef1f04..2587b15 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
@@ -20,12 +20,12 @@ package org.apache.flink.runtime.blob;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.util.FileUtils;
-
+import org.apache.flink.util.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Closeable;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -35,10 +35,17 @@ import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
- * The BLOB cache implements a local cache for content-addressable BLOBs. When requesting BLOBs through the
- * {@link BlobCache#getURL} methods, the BLOB cache will first attempt serve the file from its local cache. Only if the
- * local cache does not contain the desired BLOB, the BLOB cache will try to download it from the BLOB server.
+ * The BLOB cache implements a local cache for content-addressable BLOBs.
+ *
+ * <p>When requesting BLOBs through the {@link BlobCache#getURL} methods, the
+ * BLOB cache will first attempt to serve the file from its local cache. Only if
+ * the local cache does not contain the desired BLOB, the BLOB cache will try to
+ * download it from a distributed file system (if available) or the BLOB
+ * server.</p>
  */
 public final class BlobCache implements BlobService {
 
@@ -47,8 +54,12 @@ public final class BlobCache implements BlobService {
 
 	private final InetSocketAddress serverAddress;
 
+	/** Root directory for local file storage */
 	private final File storageDir;
 
+	/** Blob store for distributed file storage, e.g. in HA */
+	private final BlobStore blobStore;
+
 	private final AtomicBoolean shutdownRequested = new AtomicBoolean();
 
 	/** Shutdown hook thread to ensure deletion of the storage directory. */
@@ -60,15 +71,62 @@ public final class BlobCache implements BlobService {
 	/** Configuration for the blob client like ssl parameters required to connect to the blob server */
 	private final Configuration blobClientConfig;
 
+	/**
+	 * Instantiates a new BLOB cache.
+	 *
+	 * @param serverAddress
+	 * 		address of the {@link BlobServer} to use for fetching files from
+	 * @param blobClientConfig
+	 * 		global configuration
+	 *
+	 * @throws IOException
+	 * 		thrown if the (local or distributed) file storage cannot be created or
+	 * 		is not usable
+	 */
+	public BlobCache(InetSocketAddress serverAddress,
+			Configuration blobClientConfig) throws IOException {
+		this(serverAddress, blobClientConfig,
+			BlobUtils.createBlobStoreFromConfig(blobClientConfig));
+	}
 
-	public BlobCache(InetSocketAddress serverAddress, Configuration blobClientConfig) {
-		if (serverAddress == null || blobClientConfig == null) {
-			throw new NullPointerException();
-		}
-
-		this.serverAddress = serverAddress;
+	/**
+	 * Instantiates a new BLOB cache.
+	 *
+	 * @param serverAddress
+	 * 		address of the {@link BlobServer} to use for fetching files from
+	 * @param blobClientConfig
+	 * 		global configuration
+	 * 	@param haServices
+	 * 		high availability services able to create a distributed blob store
+	 *
+	 * @throws IOException
+	 * 		thrown if the (local or distributed) file storage cannot be created or
+	 * 		is not usable
+	 */
+	public BlobCache(InetSocketAddress serverAddress,
+		Configuration blobClientConfig, HighAvailabilityServices haServices) throws IOException {
+		this(serverAddress, blobClientConfig, haServices.createBlobStore());
+	}
 
-		this.blobClientConfig = blobClientConfig;
+	/**
+	 * Instantiates a new BLOB cache.
+	 *
+	 * @param serverAddress
+	 * 		address of the {@link BlobServer} to use for fetching files from
+	 * @param blobClientConfig
+	 * 		global configuration
+	 * @param blobStore
+	 * 		(distributed) blob store file system to retrieve files from first
+	 *
+	 * @throws IOException
+	 * 		thrown if the (local or distributed) file storage cannot be created or is not usable
+	 */
+	private BlobCache(
+			final InetSocketAddress serverAddress, final Configuration blobClientConfig,
+			final BlobStore blobStore) throws IOException {
+		this.serverAddress = checkNotNull(serverAddress);
+		this.blobClientConfig = checkNotNull(blobClientConfig);
+		this.blobStore = blobStore;
 
 		// configure and create the storage directory
 		String storageDirectory = blobClientConfig.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
@@ -101,92 +159,101 @@ public final class BlobCache implements BlobService {
 	 * @throws IOException Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server.
 	 */
 	public URL getURL(final BlobKey requiredBlob) throws IOException {
-		if (requiredBlob == null) {
-			throw new IllegalArgumentException("BLOB key cannot be null.");
-		}
+		checkArgument(requiredBlob != null, "BLOB key cannot be null.");
 
 		final File localJarFile = BlobUtils.getStorageLocation(storageDir, requiredBlob);
 
-		if (!localJarFile.exists()) {
+		if (localJarFile.exists()) {
+			return localJarFile.toURI().toURL();
+		}
 
-			final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE];
+		// first try the distributed blob store (if available)
+		try {
+			blobStore.get(requiredBlob, localJarFile);
+		} catch (Exception e) {
+			LOG.info("Failed to copy from blob store. Downloading from BLOB server instead.", e);
+		}
 
-			// loop over retries
-			int attempt = 0;
-			while (true) {
+		if (localJarFile.exists()) {
+			return localJarFile.toURI().toURL();
+		}
 
-				if (attempt == 0) {
-					LOG.info("Downloading {} from {}", requiredBlob, serverAddress);
-				} else {
-					LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt);
-				}
+		// fallback: download from the BlobServer
+		final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE];
 
-				try {
-					BlobClient bc = null;
-					InputStream is = null;
-					OutputStream os = null;
-
-					try {
-						bc = new BlobClient(serverAddress, blobClientConfig);
-						is = bc.get(requiredBlob);
-						os = new FileOutputStream(localJarFile);
-
-						while (true) {
-							final int read = is.read(buf);
-							if (read < 0) {
-								break;
-							}
-							os.write(buf, 0, read);
-						}
+		// loop over retries
+		int attempt = 0;
+		while (true) {
 
-						// we do explicitly not use a finally block, because we want the closing
-						// in the regular case to throw exceptions and cause the writing to fail.
-						// But, the closing on exception should not throw further exceptions and
-						// let us keep the root exception
-						os.close();
-						os = null;
-						is.close();
-						is = null;
-						bc.close();
-						bc = null;
-
-						// success, we finished
-						break;
-					}
-					catch (Throwable t) {
-						// we use "catch (Throwable)" to keep the root exception. Otherwise that exception
-						// it would be replaced by any exception thrown in the finally block
-						closeSilently(os);
-						closeSilently(is);
-						closeSilently(bc);
-
-						if (t instanceof IOException) {
-							throw (IOException) t;
-						} else {
-							throw new IOException(t.getMessage(), t);
+			if (attempt == 0) {
+				LOG.info("Downloading {} from {}", requiredBlob, serverAddress);
+			} else {
+				LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt);
+			}
+
+			try {
+				BlobClient bc = null;
+				InputStream is = null;
+				OutputStream os = null;
+
+				try {
+					bc = new BlobClient(serverAddress, blobClientConfig);
+					is = bc.get(requiredBlob);
+					os = new FileOutputStream(localJarFile);
+
+					while (true) {
+						final int read = is.read(buf);
+						if (read < 0) {
+							break;
 						}
+						os.write(buf, 0, read);
 					}
+
+					// we do explicitly not use a finally block, because we want the closing
+					// in the regular case to throw exceptions and cause the writing to fail.
+					// But, the closing on exception should not throw further exceptions and
+					// let us keep the root exception
+					os.close();
+					os = null;
+					is.close();
+					is = null;
+					bc.close();
+					bc = null;
+
+					// success, we finished
+					return localJarFile.toURI().toURL();
 				}
-				catch (IOException e) {
-					String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress +
-						" and store it under " + localJarFile.getAbsolutePath();
-					if (attempt < numFetchRetries) {
-						attempt++;
-						if (LOG.isDebugEnabled()) {
-							LOG.debug(message + " Retrying...", e);
-						} else {
-							LOG.error(message + " Retrying...");
-						}
+				catch (Throwable t) {
+					// we use "catch (Throwable)" to keep the root exception. Otherwise that exception
+					// it would be replaced by any exception thrown in the finally block
+					IOUtils.closeQuietly(os);
+					IOUtils.closeQuietly(is);
+					IOUtils.closeQuietly(bc);
+
+					if (t instanceof IOException) {
+						throw (IOException) t;
+					} else {
+						throw new IOException(t.getMessage(), t);
 					}
-					else {
-						LOG.error(message + " No retries left.", e);
-						throw new IOException(message, e);
+				}
+			}
+			catch (IOException e) {
+				String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress +
+					" and store it under " + localJarFile.getAbsolutePath();
+				if (attempt < numFetchRetries) {
+					attempt++;
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(message + " Retrying...", e);
+					} else {
+						LOG.error(message + " Retrying...");
 					}
 				}
-			} // end loop over retries
-		}
-
-		return localJarFile.toURI().toURL();
+				else {
+					LOG.error(message + " No retries left.", e);
+					throw new IOException(message, e);
+				}
+			}
+		} // end loop over retries
 	}
 
 	/**
@@ -202,18 +269,23 @@ public final class BlobCache implements BlobService {
 	}
 
 	/**
-	 * Deletes the file associated with the given key from the BLOB cache and BLOB server.
+	 * Deletes the file associated with the given key from the BLOB cache and
+	 * BLOB server.
+	 *
 	 * @param key referring to the file to be deleted
+	 * @throws IOException
+	 *         thrown if an I/O error occurs while transferring the request to
+	 *         the BLOB server or if the BLOB server cannot delete the file
 	 */
 	public void deleteGlobal(BlobKey key) throws IOException {
-		BlobClient bc = createClient();
-		try {
-			delete(key);
+		// delete locally
+		delete(key);
+		// then delete on the BLOB server
+		// (don't use the distributed storage directly - this way the blob
+		// server is aware of the delete operation, too)
+		try (BlobClient bc = createClient()) {
 			bc.delete(key);
 		}
-		finally {
-			bc.close();
-		}
 	}
 
 	@Override
@@ -258,19 +330,4 @@ public final class BlobCache implements BlobService {
 		return this.storageDir;
 	}
 
-	// ------------------------------------------------------------------------
-	//  Miscellaneous
-	// ------------------------------------------------------------------------
-
-	private void closeSilently(Closeable closeable) {
-		if (closeable != null) {
-			try {
-				closeable.close();
-			} catch (Throwable t) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Error while closing resource after BLOB transfer.", t);
-				}
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
index 2748967..ea90f54 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
@@ -590,7 +590,8 @@ public final class BlobClient implements Closeable {
 	 * @param key
 	 *        the key to identify the BLOB
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while transferring the request to the BLOB server
+	 *         thrown if an I/O error occurs while transferring the request to
+	 *         the BLOB server or if the BLOB server cannot delete the file
 	 */
 	public void delete(BlobKey key) throws IOException {
 		if (key == null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index d4190a7..5b00ae4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -21,10 +21,6 @@ package org.apache.flink.runtime.blob;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.net.SSLUtils;
@@ -49,8 +45,8 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
 
 /**
  * This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and
@@ -77,10 +73,10 @@ public class BlobServer extends Thread implements BlobService {
 	/** Indicates whether a shutdown of server component has been requested. */
 	private final AtomicBoolean shutdownRequested = new AtomicBoolean();
 
-	/** Is the root directory for file storage */
+	/** Root directory for local file storage */
 	private final File storageDir;
 
-	/** Blob store for HA */
+	/** Blob store for distributed file storage, e.g. in HA */
 	private final BlobStore blobStore;
 
 	/** Set of currently running threads */
@@ -99,10 +95,11 @@ public class BlobServer extends Thread implements BlobService {
 	 * Instantiates a new BLOB server and binds it to a free network port.
 	 *
 	 * @throws IOException
-	 *         thrown if the BLOB server cannot bind to a free network port
+	 * 		thrown if the BLOB server cannot bind to a free network port or if the
+	 * 		(local or distributed) file storage cannot be created or is not usable
 	 */
 	public BlobServer(Configuration config) throws IOException {
-		this(config, createBlobStoreFromConfig(config));
+		this(config, BlobUtils.createBlobStoreFromConfig(config));
 	}
 
 	public BlobServer(Configuration config, HighAvailabilityServices haServices) throws IOException {
@@ -110,11 +107,9 @@ public class BlobServer extends Thread implements BlobService {
 	}
 
 	private BlobServer(Configuration config, BlobStore blobStore) throws IOException {
-		checkNotNull(config);
+		this.blobServiceConfiguration = checkNotNull(config);
 		this.blobStore = checkNotNull(blobStore);
 
-		this.blobServiceConfiguration = config;
-
 		// configure and create the storage directory
 		String storageDirectory = config.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
 		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
@@ -358,9 +353,7 @@ public class BlobServer extends Thread implements BlobService {
 	 */
 	@Override
 	public URL getURL(BlobKey requiredBlob) throws IOException {
-		if (requiredBlob == null) {
-			throw new IllegalArgumentException("Required BLOB cannot be null.");
-		}
+		checkArgument(requiredBlob != null, "BLOB key cannot be null.");
 
 		final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob);
 
@@ -450,37 +443,4 @@ public class BlobServer extends Thread implements BlobService {
 		}
 	}
 
-	private static BlobStore createBlobStoreFromConfig(Configuration config) throws IOException {
-		HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
-
-		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
-		return new VoidBlobStore();
-		} else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
-			final String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
-			if (isNullOrWhitespaceOnly(storagePath)) {
-				throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
-						HighAvailabilityOptions.HA_STORAGE_PATH);
-			}
-
-			final Path path;
-			try {
-				path = new Path(storagePath);
-			} catch (Exception e) {
-				throw new IOException("Invalid path for highly available storage (" +
-						HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
-			}
-
-			final FileSystem fileSystem;
-			try {
-				fileSystem = path.getFileSystem();
-			} catch (Exception e) {
-				throw new IOException("Could not create FileSystem for highly available storage (" +
-						HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
-			}
-
-			return new FileSystemBlobStore(fileSystem, storagePath);
-		} else {
-			throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + ".");
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
index 321fc67..13a90c6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
@@ -168,6 +168,14 @@ class BlobServerConnection extends Thread {
 	 *         thrown if an I/O error occurs while reading/writing data from/to the respective streams
 	 */
 	private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) throws IOException {
+		/**
+		 * Retrieve the file from the (distributed?) BLOB store and store it
+		 * locally, then send it to the service which requested it.
+		 *
+		 * Instead, we could send it from the distributed store directly but
+		 * chances are high that if there is one request, there will be more
+		 * so a local cache makes more sense.
+		 */
 
 		File blobFile;
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
index 7050338..64dc942 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.blob;
 import org.apache.flink.api.common.JobID;
 
 import java.io.File;
+import java.io.IOException;
 
 /**
  * A blob store.
@@ -32,9 +33,9 @@ public interface BlobStore {
 	 *
 	 * @param localFile The file to copy
 	 * @param blobKey   The ID for the file in the blob store
-	 * @throws Exception If the copy fails
+	 * @throws IOException If the copy fails
 	 */
-	void put(File localFile, BlobKey blobKey) throws Exception;
+	void put(File localFile, BlobKey blobKey) throws IOException;
 
 	/**
 	 * Copies a local file to the blob store.
@@ -44,18 +45,18 @@ public interface BlobStore {
 	 * @param localFile The file to copy
 	 * @param jobId     The JobID part of ID for the file in the blob store
 	 * @param key       The String part of ID for the file in the blob store
-	 * @throws Exception If the copy fails
+	 * @throws IOException If the copy fails
 	 */
-	void put(File localFile, JobID jobId, String key) throws Exception;
+	void put(File localFile, JobID jobId, String key) throws IOException;
 
 	/**
 	 * Copies a blob to a local file.
 	 *
 	 * @param blobKey   The blob ID
 	 * @param localFile The local file to copy to
-	 * @throws Exception If the copy fails
+	 * @throws IOException If the copy fails
 	 */
-	void get(BlobKey blobKey, File localFile) throws Exception;
+	void get(BlobKey blobKey, File localFile) throws IOException;
 
 	/**
 	 * Copies a blob to a local file.
@@ -63,19 +64,23 @@ public interface BlobStore {
 	 * @param jobId     The JobID part of ID for the blob
 	 * @param key       The String part of ID for the blob
 	 * @param localFile The local file to copy to
-	 * @throws Exception If the copy fails
+	 * @throws IOException If the copy fails
 	 */
-	void get(JobID jobId, String key, File localFile) throws Exception;
+	void get(JobID jobId, String key, File localFile) throws IOException;
 
 	/**
-	 * Deletes a blob.
+	 * Tries to delete a blob from storage.
+	 *
+	 * <p>NOTE: This also tries to delete any created directories if empty.</p>
 	 *
 	 * @param blobKey The blob ID
 	 */
 	void delete(BlobKey blobKey);
 
 	/**
-	 * Deletes a blob.
+	 * Tries to delete a blob from storage.
+	 *
+	 * <p>NOTE: This also tries to delete any created directories if empty.</p>
 	 *
 	 * @param jobId The JobID part of ID for the blob
 	 * @param key   The String part of ID for the blob
@@ -83,7 +88,9 @@ public interface BlobStore {
 	void delete(JobID jobId, String key);
 
 	/**
-	 * Deletes blobs.
+	 * Tries to delete all blobs for the given job from storage.
+	 *
+	 * <p>NOTE: This also tries to delete any created directories if empty.</p>
 	 *
 	 * @param jobId The JobID part of all blobs to delete
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index aeaa602..b5ba565 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -21,20 +21,19 @@ package org.apache.flink.runtime.blob;
 import com.google.common.io.BaseEncoding;
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.IOUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.highavailability.ZookeeperHaServices;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
 
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.Socket;
-import java.net.URI;
 import java.nio.charset.Charset;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -68,11 +67,39 @@ public class BlobUtils {
 	static final Charset DEFAULT_CHARSET = Charset.forName("utf-8");
 
 	/**
+	 * Creates a BlobStore based on the parameters set in the configuration.
+	 *
+	 * @param config
+	 * 		configuration to use
+	 *
+	 * @return a (distributed) blob store for high availability
+	 *
+	 * @throws IOException
+	 * 		thrown if the (distributed) file storage cannot be created
+	 */
+	static BlobStore createBlobStoreFromConfig(Configuration config) throws IOException {
+		HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
+
+		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
+			return new VoidBlobStore();
+		} else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
+			return ZookeeperHaServices.createBlobStore(config);
+		} else {
+			throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + "'.");
+		}
+	}
+
+	/**
 	 * Creates a storage directory for a blob service.
 	 *
 	 * @return the storage directory used by a BLOB service
+	 *
+	 * @throws IOException
+	 * 		thrown if the (local or distributed) file storage cannot be created or
+	 * 		is not usable
 	 */
-	static File initStorageDirectory(String storageDirectory) {
+	static File initStorageDirectory(String storageDirectory) throws
+		IOException {
 		File baseDir;
 		if (StringUtils.isNullOrWhitespaceOnly(storageDirectory)) {
 			baseDir = new File(System.getProperty("java.io.tmpdir"));
@@ -96,7 +123,7 @@ public class BlobUtils {
 		}
 
 		// max attempts exceeded to find a storage directory
-		throw new RuntimeException("Could not create storage directory for BLOB store in '" + baseDir + "'.");
+		throw new IOException("Could not create storage directory for BLOB store in '" + baseDir + "'.");
 	}
 
 	/**
@@ -341,7 +368,7 @@ public class BlobUtils {
 	 */
 	static String getRecoveryPath(String basePath, BlobKey blobKey) {
 		// format: $base/cache/blob_$key
-		return String.format("%s/cache/%s", basePath, BLOB_FILE_PREFIX + blobKey.toString());
+		return String.format("%s/cache/%s%s", basePath, BLOB_FILE_PREFIX, blobKey.toString());
 	}
 
 	/**
@@ -353,8 +380,8 @@ public class BlobUtils {
 	 */
 	static String getRecoveryPath(String basePath, JobID jobId, String key) {
 		// format: $base/job_$id/blob_$key
-		return String.format("%s/%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString(),
-				BLOB_FILE_PREFIX + encodeKey(key));
+		return String.format("%s/%s%s/%s%s", basePath, JOB_DIR_PREFIX, jobId.toString(),
+				BLOB_FILE_PREFIX, encodeKey(key));
 	}
 
 	/**
@@ -363,33 +390,7 @@ public class BlobUtils {
 	 * <p>The returned path can be used with the state backend for recovery purposes.
 	 */
 	static String getRecoveryPath(String basePath, JobID jobId) {
-		return String.format("%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString());
-	}
-
-	/**
-	 * Copies the file from the recovery path to the local file.
-	 */
-	static void copyFromRecoveryPath(String recoveryPath, File localBlobFile) throws Exception {
-		if (recoveryPath == null) {
-			throw new IllegalStateException("Failed to determine recovery path.");
-		}
-
-		if (!localBlobFile.createNewFile()) {
-			throw new IllegalStateException("Failed to create new local file to copy to");
-		}
-
-		URI uri = new URI(recoveryPath);
-		Path path = new Path(recoveryPath);
-
-		if (FileSystem.get(uri).exists(path)) {
-			try (InputStream is = FileSystem.get(uri).open(path)) {
-				FileOutputStream fos = new FileOutputStream(localBlobFile);
-				IOUtils.copyBytes(is, fos); // closes the streams
-			}
-		}
-		else {
-			throw new IOException("Cannot find required BLOB at '" + recoveryPath + "' for recovery.");
-		}
+		return String.format("%s/%s%s", basePath, JOB_DIR_PREFIX, jobId.toString());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
index 2c05002..7cfce7a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -64,16 +64,16 @@ public class FileSystemBlobStore implements BlobStore {
 	// - Put ------------------------------------------------------------------
 
 	@Override
-	public void put(File localFile, BlobKey blobKey) throws Exception {
+	public void put(File localFile, BlobKey blobKey) throws IOException {
 		put(localFile, BlobUtils.getRecoveryPath(basePath, blobKey));
 	}
 
 	@Override
-	public void put(File localFile, JobID jobId, String key) throws Exception {
+	public void put(File localFile, JobID jobId, String key) throws IOException {
 		put(localFile, BlobUtils.getRecoveryPath(basePath, jobId, key));
 	}
 
-	private void put(File fromFile, String toBlobPath) throws Exception {
+	private void put(File fromFile, String toBlobPath) throws IOException {
 		try (OutputStream os = fileSystem.create(new Path(toBlobPath), true)) {
 			LOG.debug("Copying from {} to {}.", fromFile, toBlobPath);
 			Files.copy(fromFile, os);
@@ -83,16 +83,16 @@ public class FileSystemBlobStore implements BlobStore {
 	// - Get ------------------------------------------------------------------
 
 	@Override
-	public void get(BlobKey blobKey, File localFile) throws Exception {
+	public void get(BlobKey blobKey, File localFile) throws IOException {
 		get(BlobUtils.getRecoveryPath(basePath, blobKey), localFile);
 	}
 
 	@Override
-	public void get(JobID jobId, String key, File localFile) throws Exception {
+	public void get(JobID jobId, String key, File localFile) throws IOException {
 		get(BlobUtils.getRecoveryPath(basePath, jobId, key), localFile);
 	}
 
-	private void get(String fromBlobPath, File toFile) throws Exception {
+	private void get(String fromBlobPath, File toFile) throws IOException {
 		checkNotNull(fromBlobPath, "Blob path");
 		checkNotNull(toFile, "File");
 
@@ -102,17 +102,21 @@ public class FileSystemBlobStore implements BlobStore {
 
 		final Path fromPath = new Path(fromBlobPath);
 
-		if (fileSystem.exists(fromPath)) {
-			try (InputStream is = fileSystem.open(fromPath);
-				FileOutputStream fos = new FileOutputStream(toFile))
-			{
-				LOG.debug("Copying from {} to {}.", fromBlobPath, toFile);
-				IOUtils.copyBytes(is, fos); // closes the streams
+		boolean success = false;
+		try (InputStream is = fileSystem.open(fromPath);
+			FileOutputStream fos = new FileOutputStream(toFile)) {
+			LOG.debug("Copying from {} to {}.", fromBlobPath, toFile);
+			IOUtils.copyBytes(is, fos); // closes the streams
+			success = true;
+		} finally {
+			// if the copy fails, we need to remove the target file because
+			// outside code relies on a correct file as long as it exists
+			if (!success) {
+				try {
+					toFile.delete();
+				} catch (Throwable ignored) {}
 			}
 		}
-		else {
-			throw new IOException(fromBlobPath + " does not exist.");
-		}
 	}
 
 	// - Delete ---------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
index ece2ac1..8606844 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.blob;
 import org.apache.flink.api.common.JobID;
 
 import java.io.File;
+import java.io.IOException;
 
 /**
  * A blob store doing nothing.
@@ -28,19 +29,19 @@ import java.io.File;
 public class VoidBlobStore implements BlobStore {
 
 	@Override
-	public void put(File localFile, BlobKey blobKey) throws Exception {
+	public void put(File localFile, BlobKey blobKey) throws IOException {
 	}
 
 	@Override
-	public void put(File localFile, JobID jobId, String key) throws Exception {
+	public void put(File localFile, JobID jobId, String key) throws IOException {
 	}
 
 	@Override
-	public void get(BlobKey blobKey, File localFile) throws Exception {
+	public void get(BlobKey blobKey, File localFile) throws IOException {
 	}
 
 	@Override
-	public void get(JobID jobId, String key, File localFile) throws Exception {
+	public void get(JobID jobId, String key, File localFile) throws IOException {
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index 9f0c573..76d6d86 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -209,7 +209,13 @@ public class JobClient {
 			Option<String> jmHost = jobManager.actor().path().address().host();
 			String jmHostname = jmHost.isDefined() ? jmHost.get() : "localhost";
 			InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, props.blobManagerPort());
-			final BlobCache blobClient = new BlobCache(serverAddress, config);
+			final BlobCache blobClient;
+			try {
+				blobClient = new BlobCache(serverAddress, config);
+			} catch (IOException e) {
+				throw new JobRetrievalException(jobID,
+					"Failed to setup blob cache", e);
+			}
 
 			final Collection<BlobKey> requiredJarFiles = props.requiredJarFiles();
 			final Collection<URL> requiredClasspaths = props.requiredClasspaths();

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
index c94768d..b0d5d83 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
@@ -37,11 +37,12 @@ import org.apache.flink.runtime.blob.BlobService;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * For each job graph that is submitted to the system the library cache manager maintains
  * a set of libraries (typically JAR files) which the job requires to run. The library cache manager
@@ -73,7 +74,7 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC
 	// --------------------------------------------------------------------------------------------
 
 	public BlobLibraryCacheManager(BlobService blobService, long cleanupInterval) {
-		this.blobService = blobService;
+		this.blobService = checkNotNull(blobService);
 
 		// Initializing the clean up task
 		this.cleanupTimer = new Timer(true);
@@ -91,8 +92,8 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC
 	@Override
 	public void registerTask(JobID jobId, ExecutionAttemptID task, Collection<BlobKey> requiredJarFiles,
 			Collection<URL> requiredClasspaths) throws IOException {
-		Preconditions.checkNotNull(jobId, "The JobId must not be null.");
-		Preconditions.checkNotNull(task, "The task execution id must not be null.");
+		checkNotNull(jobId, "The JobId must not be null.");
+		checkNotNull(task, "The task execution id must not be null.");
 
 		if (requiredJarFiles == null) {
 			requiredJarFiles = Collections.emptySet();
@@ -153,8 +154,8 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC
 	
 	@Override
 	public void unregisterTask(JobID jobId, ExecutionAttemptID task) {
-		Preconditions.checkNotNull(jobId, "The JobId must not be null.");
-		Preconditions.checkNotNull(task, "The task execution id must not be null.");
+		checkNotNull(jobId, "The JobId must not be null.");
+		checkNotNull(task, "The task execution id must not be null.");
 
 		synchronized (lockObject) {
 			LibraryCacheEntry entry = cacheEntries.get(jobId);

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
index 25d21ef..ed0ad17 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
@@ -154,7 +154,21 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 
 	@Override
 	public BlobStore createBlobStore() throws IOException {
-		final String storagePath = configuration.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+		return createBlobStore(configuration);
+	}
+
+	/**
+	 * Creates the BLOB store in which BLOBs are stored in a highly-available
+	 * fashion.
+	 *
+	 * @param configuration configuration to extract the storage path from
+	 * @return Blob store
+	 * @throws IOException if the blob store could not be created
+	 */
+	public static BlobStore createBlobStore(
+		final Configuration configuration) throws IOException {
+		String storagePath = configuration.getValue(
+			HighAvailabilityOptions.HA_STORAGE_PATH);
 		if (isNullOrWhitespaceOnly(storagePath)) {
 			throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
 					HighAvailabilityOptions.HA_STORAGE_PATH);
@@ -176,6 +190,10 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
 		}
 
+		final String clusterId =
+			configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
+		storagePath += "/" + clusterId;
+
 		return new FileSystemBlobStore(fileSystem, storagePath);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index f11cb98..58bbfac 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -794,11 +794,20 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 		InetSocketAddress address = new InetSocketAddress(jobMasterGateway.getAddress(), blobPort);
 
-		BlobCache blobCache = new BlobCache(address, taskManagerConfiguration.getConfiguration());
-
-		LibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(
-			blobCache,
-			taskManagerConfiguration.getCleanupInterval());
+		final LibraryCacheManager libraryCacheManager;
+		try {
+			final BlobCache blobCache = new BlobCache(address, taskManagerConfiguration.getConfiguration(), haServices);
+			libraryCacheManager = new BlobLibraryCacheManager(
+				blobCache,
+				taskManagerConfiguration.getCleanupInterval());
+		} catch (IOException e) {
+			// Can't pass the IOException up - we need a RuntimeException anyway
+			// two levels up where this is run asynchronously. Also, we don't
+			// know whether this is caught in the thread running this method.
+			final String message = "Could not create BLOB cache or library cache.";
+			log.error(message, e);
+			throw new RuntimeException(message, e);
+		}
 
 		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier(
 			jobManagerLeaderId,

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
index 4aa9a21..34a8a39 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
@@ -19,7 +19,10 @@
 package org.apache.flink.runtime.blob;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -33,22 +36,52 @@ import static org.junit.Assert.*;
  */
 public class BlobCacheRetriesTest {
 
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
 	/**
 	 * A test where the connection fails twice and then the get operation succeeds.
 	 */
 	@Test
 	public void testBlobFetchRetries() {
+		final Configuration config = new Configuration();
+
+		testBlobFetchRetries(config);
+	}
 
+	/**
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (with high availability set).
+	 */
+	@Test
+	public void testBlobFetchRetriesHa() {
+		final Configuration config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+			temporaryFolder.getRoot().getPath());
+
+		testBlobFetchRetries(config);
+	}
+
+	/**
+	 * A test where the BlobCache must use the BlobServer and the connection
+	 * fails twice and then the get operation succeeds.
+	 *
+	 * @param config
+	 * 		configuration to use (the BlobCache will get some additional settings
+	 * 		set compared to this one)
+	 */
+	private void testBlobFetchRetries(final Configuration config) {
 		final byte[] data = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0};
 
 		BlobServer server = null;
 		BlobCache cache = null;
 		try {
-			final Configuration config = new Configuration();
 
 			server = new TestingFailingBlobServer(config, 2);
 
-			final InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort());
+			final InetSocketAddress
+				serverAddress = new InetSocketAddress("localhost", server.getPort());
 
 			// upload some blob
 			BlobClient blobClient = null;
@@ -64,9 +97,15 @@ public class BlobCacheRetriesTest {
 				}
 			}
 
-			cache = new BlobCache(serverAddress, config);
+			// create a separate config for the cache with no access to
+			// the (shared) storage path if available so that the cache
+			// will always bother the BlobServer!
+			final Configuration cacheConfig = new Configuration(config);
+			cacheConfig.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+				temporaryFolder.getRoot().getPath() + "/does-not-exist");
+			cache = new BlobCache(serverAddress, cacheConfig);
 
-			// trigger a download - it should fail on the first time, but retry, and succeed at the second time
+			// trigger a download - it should fail the first two times, but retry, and succeed eventually
 			URL url = cache.getURL(key);
 			InputStream is = url.openStream();
 			try {
@@ -97,17 +136,44 @@ public class BlobCacheRetriesTest {
 	 */
 	@Test
 	public void testBlobFetchWithTooManyFailures() {
+		final Configuration config = new Configuration();
 
+		testBlobFetchWithTooManyFailures(config);
+	}
+
+	/**
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (with high availability set).
+	 */
+	@Test
+	public void testBlobFetchWithTooManyFailuresHa() {
+		final Configuration config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+			temporaryFolder.getRoot().getPath());
+
+		testBlobFetchWithTooManyFailures(config);
+	}
+
+	/**
+	 * A test where the BlobCache must use the BlobServer and the connection
+	 * fails too often which eventually fails the GET request.
+	 *
+	 * @param config
+	 * 		configuration to use (the BlobCache will get some additional settings
+	 * 		set compared to this one)
+	 */
+	private void testBlobFetchWithTooManyFailures(final Configuration config) {
 		final byte[] data = new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 0 };
 
 		BlobServer server = null;
 		BlobCache cache = null;
 		try {
-			final Configuration config = new Configuration();
 
 			server = new TestingFailingBlobServer(config, 10);
 
-			final InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort());
+			final InetSocketAddress
+				serverAddress = new InetSocketAddress("localhost", server.getPort());
 
 			// upload some blob
 			BlobClient blobClient = null;
@@ -123,7 +189,13 @@ public class BlobCacheRetriesTest {
 				}
 			}
 
-			cache = new BlobCache(serverAddress, config);
+			// create a separate config for the cache with no access to
+			// the (shared) storage path if available so that the cache
+			// will always bother the BlobServer!
+			final Configuration cacheConfig = new Configuration(config);
+			cacheConfig.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+				temporaryFolder.getRoot().getPath() + "/does-not-exist");
+			cache = new BlobCache(serverAddress, cacheConfig);
 
 			// trigger a download - it should fail eventually
 			try {

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java
index 7ba5a8a..db55331 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java
@@ -18,6 +18,12 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
 import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
@@ -25,9 +31,6 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.configuration.Configuration;
-import org.junit.Test;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -38,9 +41,48 @@ import static org.junit.Assert.fail;
  */
 public class BlobCacheSuccessTest {
 
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	/**
+	 * BlobCache with no HA. BLOBs need to be downloaded form a working
+	 * BlobServer.
+	 */
 	@Test
 	public void testBlobCache() {
+		Configuration config = new Configuration();
+		uploadFileGetTest(config, false, false);
+	}
+
+	/**
+	 * BlobCache is configured in HA mode and the cache can download files from
+	 * the file system directly and does not need to download BLOBs from the
+	 * BlobServer.
+	 */
+	@Test
+	public void testBlobCacheHa() {
+		Configuration config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+			temporaryFolder.getRoot().getPath());
+		uploadFileGetTest(config, true, true);
+	}
 
+	/**
+	 * BlobCache is configured in HA mode but the cache itself cannot access the
+	 * file system and thus needs to download BLOBs from the BlobServer.
+	 */
+	@Test
+	public void testBlobCacheHaFallback() {
+		Configuration config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+			temporaryFolder.getRoot().getPath());
+		uploadFileGetTest(config, false, false);
+	}
+
+	private void uploadFileGetTest(final Configuration config, boolean cacheWorksWithoutServer,
+		boolean cacheHasAccessToFs) {
 		// First create two BLOBs and upload them to BLOB server
 		final byte[] buf = new byte[128];
 		final List<BlobKey> blobKeys = new ArrayList<BlobKey>(2);
@@ -50,7 +92,6 @@ public class BlobCacheSuccessTest {
 		try {
 
 			// Start the BLOB server
-			Configuration config = new Configuration();
 			blobServer = new BlobServer(config);
 			final InetSocketAddress serverAddress = new InetSocketAddress(blobServer.getPort());
 
@@ -69,15 +110,34 @@ public class BlobCacheSuccessTest {
 				}
 			}
 
-			blobCache = new BlobCache(serverAddress, new Configuration());
+			if (cacheWorksWithoutServer) {
+				// Now, shut down the BLOB server, the BLOBs must still be accessible through the cache.
+				blobServer.shutdown();
+				blobServer = null;
+			}
+
+			final Configuration cacheConfig;
+			if (cacheHasAccessToFs) {
+				cacheConfig = config;
+			} else {
+				// just in case parameters are still read from the server,
+				// create a separate configuration object for the cache
+				cacheConfig = new Configuration(config);
+				cacheConfig.setString(HighAvailabilityOptions.HA_STORAGE_PATH,
+					temporaryFolder.getRoot().getPath() + "/does-not-exist");
+			}
+
+			blobCache = new BlobCache(serverAddress, cacheConfig);
 
 			for (BlobKey blobKey : blobKeys) {
 				blobCache.getURL(blobKey);
 			}
 
-			// Now, shut down the BLOB server, the BLOBs must still be accessible through the cache.
-			blobServer.shutdown();
-			blobServer = null;
+			if (blobServer != null) {
+				// Now, shut down the BLOB server, the BLOBs must still be accessible through the cache.
+				blobServer.shutdown();
+				blobServer = null;
+			}
 
 			final URL[] urls = new URL[blobKeys.size()];
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
index a8eb1d3..d043665 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.blob;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -26,9 +25,9 @@ import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.junit.After;
-import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.io.IOException;
@@ -39,27 +38,14 @@ import java.util.Arrays;
 import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class BlobRecoveryITCase {
 
-	private File recoveryDir;
-
-	@Before
-	public void setUp() throws Exception {
-		recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir");
-		if (!recoveryDir.exists() && !recoveryDir.mkdirs()) {
-			throw new IllegalStateException("Failed to create temp directory for test");
-		}
-	}
-
-	@After
-	public void cleanUp() throws Exception {
-		if (recoveryDir != null) {
-			FileUtils.deleteDirectory(recoveryDir);
-		}
-	}
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
 	/**
 	 * Tests that with {@link HighAvailabilityMode#ZOOKEEPER} distributed JARs are recoverable from any
@@ -70,13 +56,14 @@ public class BlobRecoveryITCase {
 		Configuration config = new Configuration();
 		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
 		config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, recoveryDir.getPath());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.getRoot().getPath());
 
 		testBlobServerRecovery(config);
 	}
 
 	public static void testBlobServerRecovery(final Configuration config) throws IOException {
-		String storagePath = config.getString(HighAvailabilityOptions.HA_STORAGE_PATH);
+		final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID);
+		String storagePath = config.getString(HighAvailabilityOptions.HA_STORAGE_PATH) + "/" + clusterId;
 		Random rand = new Random();
 
 		BlobServer[] server = new BlobServer[2];
@@ -84,7 +71,6 @@ public class BlobRecoveryITCase {
 		BlobClient client = null;
 
 		try {
-
 			for (int i = 0; i < server.length; i++) {
 				server[i] = new BlobServer(config);
 				serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort());
@@ -165,6 +151,7 @@ public class BlobRecoveryITCase {
 			client.delete(jobId[1], testKey[1]);
 
 			// Verify everything is clean
+			assertTrue("HA storage directory does not exist", fs.exists(new Path(storagePath)));
 			if (fs.exists(blobServerPath)) {
 				final org.apache.flink.core.fs.FileStatus[] recoveryFiles =
 					fs.listStatus(blobServerPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
index 53e1d73..025a2ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
@@ -85,16 +85,7 @@ public class BlobServerDeleteTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (client != null) {
-				try {
-					client.close();
-				} catch (Throwable t) {
-					t.printStackTrace();
-				}
-			}
-			if (server != null) {
-				server.shutdown();
-			}
+			cleanup(server, client);
 		}
 	}
 
@@ -157,16 +148,7 @@ public class BlobServerDeleteTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (client != null) {
-				try {
-					client.close();
-				} catch (Throwable t) {
-					t.printStackTrace();
-				}
-			}
-			if (server != null) {
-				server.shutdown();
-			}
+			cleanup(server, client);
 		}
 	}
 
@@ -205,16 +187,7 @@ public class BlobServerDeleteTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (client != null) {
-				try {
-					client.close();
-				} catch (Throwable t) {
-					t.printStackTrace();
-				}
-			}
-			if (server != null) {
-				server.shutdown();
-			}
+			cleanup(server, client);
 		}
 	}
 
@@ -254,16 +227,7 @@ public class BlobServerDeleteTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (client != null) {
-				try {
-					client.close();
-				} catch (Throwable t) {
-					t.printStackTrace();
-				}
-			}
-			if (server != null) {
-				server.shutdown();
-			}
+			cleanup(server, client);
 		}
 	}
 
@@ -312,16 +276,20 @@ public class BlobServerDeleteTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (client != null) {
-				try {
-					client.close();
-				} catch (Throwable t) {
-					t.printStackTrace();
-				}
-			}
-			if (server != null) {
-				server.shutdown();
+			cleanup(server, client);
+		}
+	}
+
+	private void cleanup(BlobServer server, BlobClient client) {
+		if (client != null) {
+			try {
+				client.close();
+			} catch (Throwable t) {
+				t.printStackTrace();
 			}
 		}
+		if (server != null) {
+			server.shutdown();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
index 36ae8cc..ea0eb94 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
@@ -40,6 +40,7 @@ public class BlobServerRangeTest extends TestLogger {
 		Configuration conf = new Configuration();
 		conf.setString(ConfigConstants.BLOB_SERVER_PORT, "0");
 		BlobServer srv = new BlobServer(conf);
+		srv.shutdown();
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java
index 63ec338..081e28c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java
@@ -30,6 +30,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
+import java.io.IOException;
 
 public class BlobUtilsTest {
 
@@ -55,8 +56,9 @@ public class BlobUtilsTest {
 		assertTrue(blobUtilsTestDirectory.delete());
 	}
 
-	@Test(expected = Exception.class)
-	public void testExceptionOnCreateStorageDirectoryFailure() {
+	@Test(expected = IOException.class)
+	public void testExceptionOnCreateStorageDirectoryFailure() throws
+		IOException {
 		// Should throw an Exception
 		BlobUtils.initStorageDirectory(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS).getAbsolutePath());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9f544d83/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
index a727d51..d3925be 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
@@ -43,6 +43,7 @@ import java.util.List;
 import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 public class BlobLibraryCacheRecoveryITCase {
 
@@ -62,12 +63,12 @@ public class BlobLibraryCacheRecoveryITCase {
 		BlobCache cache = null;
 		BlobLibraryCacheManager libCache = null;
 
-		try {
-			Configuration config = new Configuration();
-			config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
-			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
+		Configuration config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
 
+		try {
 			for (int i = 0; i < server.length; i++) {
 				server[i] = new BlobServer(config);
 				serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort());
@@ -144,8 +145,11 @@ public class BlobLibraryCacheRecoveryITCase {
 				client.delete(keys.get(1));
 			}
 
-			// Verify everything is clean
-			File[] recoveryFiles = temporaryFolder.getRoot().listFiles();
+			// Verify everything is clean below recoveryDir/<cluster_id>
+			final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID);
+			File haBlobStoreDir = new File(temporaryFolder.getRoot(), clusterId);
+			File[] recoveryFiles = haBlobStoreDir.listFiles();
+			assertNotNull("HA storage directory does not exist", recoveryFiles);
 			assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length);
 		}
 		finally {


[06/19] flink git commit: [FLINK-5828] [distributed runtime] Fix initialization of Blob storage directories

Posted by se...@apache.org.
[FLINK-5828] [distributed runtime] Fix initialization of Blob storage directories

Flip the logic (check existence and create directory) to resolve currency problem

This closes #3342


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

Branch: refs/heads/master
Commit: 20420fc6ee153c7171265dda7bf7d593c17fb375
Parents: 70475b3
Author: \u58eb\u8fdc <bo...@alipay.com>
Authored: Fri Feb 17 17:42:22 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/flink/runtime/blob/BlobUtils.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/20420fc6/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index 136df09..aeaa602 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -124,7 +124,7 @@ public class BlobUtils {
 	private static File getCacheDirectory(File storageDir) {
 		final File cacheDirectory = new File(storageDir, "cache");
 
-		if (!cacheDirectory.exists() && !cacheDirectory.mkdirs()) {
+		if (!cacheDirectory.mkdirs() && !cacheDirectory.exists()) {
 			throw new RuntimeException("Could not create cache directory '" + cacheDirectory.getAbsolutePath() + "'.");
 		}
 


[15/19] flink git commit: [FLINK-5747] [distributed coordination] Eager scheduling allocates slots and deploys tasks in bulk

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
index 8a9a4ce..be26e7b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
@@ -19,16 +19,13 @@
 package org.apache.flink.runtime.leaderelection;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.JobStatusListener;
+import org.apache.flink.runtime.executiongraph.TerminalJobStatusListener;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
@@ -113,7 +110,7 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 
 		ExecutionGraph executionGraph = (ExecutionGraph) ((TestingJobManagerMessages.ExecutionGraphFound) responseExecutionGraph).executionGraph();
 
-		TestJobStatusListener testListener = new TestJobStatusListener();
+		TerminalJobStatusListener testListener = new TerminalJobStatusListener();
 		executionGraph.registerJobStatusListener(testListener);
 
 		cluster.revokeLeadership();
@@ -146,20 +143,4 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 
 		return jobGraph;
 	}
-
-	public static class TestJobStatusListener implements JobStatusListener {
-
-		private final OneShotLatch terminalStateLatch = new OneShotLatch();
-
-		public void waitForTerminalState(long timeoutMillis) throws InterruptedException, TimeoutException {
-			terminalStateLatch.await(timeoutMillis, TimeUnit.MILLISECONDS);
-		}
-
-		@Override
-		public void jobStatusChanges(JobID jobId, JobStatus newJobStatus, long timestamp, Throwable error) {
-			if (newJobStatus.isGloballyTerminalState() || newJobStatus == JobStatus.SUSPENDED) {
-				terminalStateLatch.trigger();
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index d9a1896..f656622 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.util.TestLogger;
 
@@ -46,7 +47,13 @@ public class MiniClusterITCase extends TestLogger {
 		cfg.setUseSingleRpcService();
 
 		MiniCluster miniCluster = new MiniCluster(cfg);
-		executeJob(miniCluster);
+		try {
+			miniCluster.start();
+			executeJob(miniCluster);
+		}
+		finally {
+			miniCluster.shutdown();
+		}
 	}
 
 	@Test
@@ -55,7 +62,13 @@ public class MiniClusterITCase extends TestLogger {
 		cfg.setUseRpcServicePerComponent();
 
 		MiniCluster miniCluster = new MiniCluster(cfg);
-		executeJob(miniCluster);
+		try {
+			miniCluster.start();
+			executeJob(miniCluster);
+		}
+		finally {
+			miniCluster.shutdown();
+		}
 	}
 
 	@Test
@@ -64,7 +77,13 @@ public class MiniClusterITCase extends TestLogger {
 		cfg.setNumJobManagers(3);
 
 		MiniCluster miniCluster = new MiniCluster(cfg);
-		executeJob(miniCluster);
+		try {
+			miniCluster.start();
+			executeJob(miniCluster);
+		}
+		finally {
+			miniCluster.shutdown();
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -72,8 +91,6 @@ public class MiniClusterITCase extends TestLogger {
 	// ------------------------------------------------------------------------
 
 	private static void executeJob(MiniCluster miniCluster) throws Exception {
-		miniCluster.start();
-
 		JobGraph job = getSimpleJob();
 		miniCluster.runJobBlocking(job);
 	}
@@ -86,6 +103,7 @@ public class MiniClusterITCase extends TestLogger {
 
 		JobGraph jg = new JobGraph(new JobID(), "Test Job", task);
 		jg.setAllowQueuedScheduling(true);
+		jg.setScheduleMode(ScheduleMode.EAGER);
 
 		ExecutionConfig executionConfig = new ExecutionConfig();
 		executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
index 2007d35..63dc35d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.api.environment;
 
-import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  * <p>When this environment is instantiated, it uses a default parallelism of {@code 1}. The default
  * parallelism can be set via {@link #setParallelism(int)}.
  */
-@Public
+@Internal
 public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
 
 	private static final Logger LOG = LoggerFactory.getLogger(Flip6LocalStreamEnvironment.class);


[14/19] flink git commit: [FLINK-5812] [core] Cleanups in FileSystem (round 1)

Posted by se...@apache.org.
[FLINK-5812] [core] Cleanups in FileSystem (round 1)

  - This makes the FileSystem use the 'WriteMode' (otherwise it was an unused enumeration)
  - Extends comments
  - Deprecate the method that controls the replication factor and block size


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

Branch: refs/heads/master
Commit: a1bfae95fec8d076ef90d5a36ffa32d3870870d8
Parents: 31c26e3
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Feb 15 17:10:53 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:24 2017 +0100

----------------------------------------------------------------------
 .../flink/api/common/io/FileOutputFormat.java   |  4 +-
 .../org/apache/flink/core/fs/FileSystem.java    | 82 +++++++++++++++++---
 .../core/fs/SafetyNetWrapperFileSystem.java     |  6 +-
 .../flink/core/fs/local/LocalFileSystem.java    | 26 ++++---
 .../flink/util/AbstractCloseableRegistry.java   | 15 ++--
 .../flink/runtime/fs/hdfs/HadoopFileSystem.java |  4 +-
 .../flink/runtime/fs/maprfs/MapRFileSystem.java |  4 +-
 7 files changed, 104 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
index 0ab12df..1382f06 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
@@ -104,7 +104,7 @@ public abstract class FileOutputFormat<IT> extends RichOutputFormat<IT> implemen
 	protected Path outputFilePath;
 	
 	/**
-	 * The write mode of the output.	
+	 * The write mode of the output.
 	 */
 	private WriteMode writeMode;
 	
@@ -249,7 +249,7 @@ public abstract class FileOutputFormat<IT> extends RichOutputFormat<IT> implemen
 		this.actualFilePath = (numTasks > 1 || outputDirectoryMode == OutputDirectoryMode.ALWAYS) ? p.suffix("/" + getDirectoryFileName(taskNumber)) : p;
 
 		// create output file
-		this.stream = fs.create(this.actualFilePath, writeMode == WriteMode.OVERWRITE);
+		this.stream = fs.create(this.actualFilePath, writeMode);
 		
 		// at this point, the file creation must have succeeded, or an exception has been thrown
 		this.fileCreated = true;

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/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 c3828fb..4149d5e 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
@@ -33,6 +33,7 @@ import org.apache.flink.core.fs.local.LocalFileSystem;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.OperatingSystem;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,7 +60,20 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * machine-local file system). Other file system types are accessed by an implementation that bridges
  * to the suite of file systems supported by Hadoop (such as for example HDFS).
  * 
- * <h2>Data Persistence</h2>
+ * <h2>Scope and Purpose</h2>
+ * 
+ * The purpose of this abstraction is used to expose a common and well defined interface for
+ * access to files. This abstraction is used both by Flink's fault tolerance mechanism (storing
+ * state and recovery data) and by reusable built-in connectors (file sources / sinks).
+ * 
+ * <p>The purpose of this abstraction is <b>not</b> to give user programs an abstraction with
+ * extreme flexibility and control across all possible file systems. That mission would be a folly,
+ * as the differences in characteristics of even the most common file systems are already quite
+ * large. It is expected that user programs that need specialized functionality of certain file systems
+ * in their functions, operations, sources, or sinks instantiate the specialized file system adapters
+ * directly.
+ * 
+ * <h2>Data Persistence Contract</h2>
  * 
  * The FileSystem's {@link FSDataOutputStream output streams} are used to persistently store data,
  * both for results of streaming applications and for fault tolerance and recovery. It is therefore
@@ -152,6 +166,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * in between read or write operations, because there are no guarantees about the visibility of
  * operations across threads (many operations do not create memory fences).
  * 
+ * <h2>Streams Safety Net</h2>
+ * 
+ * When application code obtains a FileSystem (via {@link FileSystem#get(URI)} or via
+ * {@link Path#getFileSystem()}), the FileSystem instantiates a safety net for that FileSystem.
+ * The safety net ensures that all streams created from the FileSystem are closed when the
+ * application task finishes (or is canceled or failed). That way, the task's threads do not
+ * leak connections.
+ * 
  * @see FSDataInputStream
  * @see FSDataOutputStream
  */
@@ -164,11 +186,13 @@ public abstract class FileSystem {
 	 */
 	public enum WriteMode {
 
-		/** Creates the target file if it does not exist. Does not overwrite existing files and directories. */
+		/** Creates the target file only if no file exists at that path already.
+		 * Does not overwrite existing files and directories. */
 		NO_OVERWRITE,
 
-		/** Creates a new target file regardless of any existing files or directories. Existing files and
-		 * directories will be removed/overwritten. */
+		/** Creates a new target file regardless of any existing files or directories.
+		 * Existing files and directories will be deleted (recursively) automatically before
+		 * creating the new file. */
 		OVERWRITE
 	}
 
@@ -555,7 +579,6 @@ public abstract class FileSystem {
 	 *        source file
 	 */
 	public boolean exists(final Path f) throws IOException {
-
 		try {
 			return (getFileStatus(f) != null);
 		} catch (FileNotFoundException e) {
@@ -590,6 +613,11 @@ public abstract class FileSystem {
 
 	/**
 	 * Opens an FSDataOutputStream at the indicated Path.
+	 * 
+	 * <p>This method is deprecated, because most of its parameters are ignored by most file systems.
+	 * To control for example the replication factor and block size in the Hadoop Distributed File system,
+	 * make sure that the respective Hadoop configuration file is either linked from the Flink configuration,
+	 * or in the classpath of either Flink or the user code.
 	 *
 	 * @param f
 	 *        the file name to open
@@ -602,8 +630,15 @@ public abstract class FileSystem {
 	 *        required block replication for the file.
 	 * @param blockSize
 	 *        the size of the file blocks
-	 * @throws IOException
+	 * 
+	 * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because
+	 *                     a file already exists at that path and the write mode indicates to not
+	 *                     overwrite the file.
+	 * 
+	 * @deprecated Deprecated because not well supported across types of file systems.
+	 *             Control the behavior of specific file systems via configurations instead. 
 	 */
+	@Deprecated
 	public abstract FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
 			long blockSize) throws IOException;
 
@@ -615,9 +650,34 @@ public abstract class FileSystem {
 	 * @param overwrite
 	 *        if a file with this name already exists, then if true,
 	 *        the file will be overwritten, and if false an error will be thrown.
-	 * @throws IOException
+	 * 
+	 * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because
+	 *                     a file already exists at that path and the write mode indicates to not
+	 *                     overwrite the file.
+	 * 
+	 * @deprecated Use {@link #create(Path, WriteMode)} instead.
+	 */
+	@Deprecated
+	public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
+		return create(f, overwrite ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE);
+	}
+
+	/**
+	 * Opens an FSDataOutputStream to a new file at the given path.
+	 * 
+	 * <p>If the file already exists, the behavior depends on the given {@code WriteMode}.
+	 * If the mode is set to {@link WriteMode#NO_OVERWRITE}, then this method fails with an
+	 * exception.
+	 *
+	 * @param f The file path to write to
+	 * @param overwriteMode The action to take if a file or directory already exists at the given path.
+	 * @return The stream to the new file at the target path.
+	 * 
+	 * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because
+	 *                     a file already exists at that path and the write mode indicates to not
+	 *                     overwrite the file.
 	 */
-	public abstract FSDataOutputStream create(Path f, boolean overwrite) throws IOException;
+	public abstract FSDataOutputStream create(Path f, WriteMode overwriteMode) throws IOException;
 
 	/**
 	 * Renames the file/directory src to dst.
@@ -632,7 +692,9 @@ public abstract class FileSystem {
 	public abstract boolean rename(Path src, Path dst) throws IOException;
 
 	/**
-	 * Returns true if this is a distributed file system, false otherwise.
+	 * Returns true if this is a distributed file system. A distributed file system here means
+	 * that the file system is shared among all Flink processes that participate in a cluster or
+	 * job and that all these processes can see the same files.
 	 *
 	 * @return True, if this is a distributed file system, false otherwise.
 	 */
@@ -911,7 +973,7 @@ public abstract class FileSystem {
 	 * An identifier of a file system, via its scheme and its authority.
 	 * This class needs to stay public, because it is detected as part of the public API.
 	 */
-	public static class FSKey {
+	private static final class FSKey {
 
 		/** The scheme of the file system. */
 		private final String scheme;

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
index 63e6253..1dacafd 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
@@ -30,8 +30,8 @@ import java.net.URI;
  * {@link ClosingFSDataInputStream} or {@link ClosingFSDataOutputStream} and (ii) registers them to
  * a {@link SafetyNetCloseableRegistry}.
  *
- * Streams obtained by this are therefore managed by the {@link SafetyNetCloseableRegistry} to prevent resource leaks
- * from unclosed streams.
+ * <p>Streams obtained by this are therefore managed by the {@link SafetyNetCloseableRegistry} to
+ * prevent resource leaks from unclosed streams.
  */
 @Internal
 public class SafetyNetWrapperFileSystem extends FileSystem implements WrappingProxy<FileSystem> {
@@ -120,7 +120,7 @@ public class SafetyNetWrapperFileSystem extends FileSystem implements WrappingPr
 	}
 
 	@Override
-	public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
+	public FSDataOutputStream create(Path f, WriteMode overwrite) throws IOException {
 		FSDataOutputStream innerStream = unsafeFileSystem.create(f, overwrite);
 		return ClosingFSDataOutputStream.wrapSafe(innerStream, registry, String.valueOf(f));
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java
index acbf814..12aeb7f 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java
@@ -43,9 +43,12 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
+import java.nio.file.FileAlreadyExistsException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * The class <code>LocalFile</code> provides an implementation of the {@link FileSystem} interface
+ * The class {@code LocalFileSystem} is an implementation of the {@link FileSystem} interface
  * for the local file system of the machine where the JVM runs.
  */
 @Internal
@@ -231,28 +234,27 @@ public class LocalFileSystem extends FileSystem {
 		return (parent == null || mkdirs(parent)) && (p2f.mkdir() || p2f.isDirectory());
 	}
 
-
 	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
-			final short replication, final long blockSize) throws IOException {
+	public FSDataOutputStream create(final Path filePath, final WriteMode overwrite) throws IOException {
+		checkNotNull(filePath, "filePath");
 
-		if (exists(f) && !overwrite) {
-			throw new IOException("File already exists:" + f);
+		if (exists(filePath) && overwrite == WriteMode.NO_OVERWRITE) {
+			throw new FileAlreadyExistsException("File already exists: " + filePath);
 		}
 
-		final Path parent = f.getParent();
+		final Path parent = filePath.getParent();
 		if (parent != null && !mkdirs(parent)) {
-			throw new IOException("Mkdirs failed to create " + parent.toString());
+			throw new IOException("Mkdirs failed to create " + parent);
 		}
 
-		final File file = pathToFile(f);
+		final File file = pathToFile(filePath);
 		return new LocalDataOutputStream(file);
 	}
 
-
 	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
-		return create(f, overwrite, 0, (short) 0, 0);
+	public FSDataOutputStream create(
+			Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException {
+		return create(f, overwrite ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
index e165d97..2b7a8c8 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
@@ -27,10 +27,10 @@ import java.util.Map;
 /**
  * This is the abstract base class for registries that allow to register instances of {@link Closeable}, which are all
  * closed if this registry is closed.
- * <p>
- * Registering to an already closed registry will throw an exception and close the provided {@link Closeable}
- * <p>
- * All methods in this class are thread-safe.
+ * 
+ * <p>Registering to an already closed registry will throw an exception and close the provided {@link Closeable}
+ * 
+ * <p>All methods in this class are thread-safe.
  *
  * @param <C> Type of the closeable this registers
  * @param <T> Type for potential meta data associated with the registering closeables
@@ -51,7 +51,7 @@ public abstract class AbstractCloseableRegistry<C extends Closeable, T> implemen
 	 * {@link IllegalStateException} and closes the passed {@link Closeable}.
 	 *
 	 * @param closeable Closeable tor register
-	 * @return true if the the Closeable was newly added to the registry
+	 * 
 	 * @throws IOException exception when the registry was closed before
 	 */
 	public final void registerClosable(C closeable) throws IOException {
@@ -74,7 +74,6 @@ public abstract class AbstractCloseableRegistry<C extends Closeable, T> implemen
 	 * Removes a {@link Closeable} from the registry.
 	 *
 	 * @param closeable instance to remove from the registry.
-	 * @return true, if the instance was actually registered and now removed
 	 */
 	public final void unregisterClosable(C closeable) {
 
@@ -109,6 +108,10 @@ public abstract class AbstractCloseableRegistry<C extends Closeable, T> implemen
 		return closeableToRef;
 	}
 
+	// ------------------------------------------------------------------------
+	//  
+	// ------------------------------------------------------------------------
+
 	protected abstract void doUnRegister(C closeable, Map<Closeable, T> closeableMap);
 
 	protected abstract void doRegister(C closeable, Map<Closeable, T> closeableMap) throws IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
index 36dfa55..1371d21 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
@@ -417,9 +417,9 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 
 
 	@Override
-	public HadoopDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
+	public HadoopDataOutputStream create(final Path f, final WriteMode overwrite) throws IOException {
 		final org.apache.hadoop.fs.FSDataOutputStream fsDataOutputStream = this.fs
-			.create(new org.apache.hadoop.fs.Path(f.toString()), overwrite);
+			.create(new org.apache.hadoop.fs.Path(f.toString()), overwrite == WriteMode.OVERWRITE);
 		return new HadoopDataOutputStream(fsDataOutputStream);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1bfae95/flink-runtime/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java
index a7ef441..57eea6f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java
@@ -327,11 +327,11 @@ public final class MapRFileSystem extends FileSystem {
 	}
 
 	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite)
+	public FSDataOutputStream create(final Path f, final WriteMode overwrite)
 			throws IOException {
 
 		final org.apache.hadoop.fs.FSDataOutputStream fdos = this.fs.create(
-				new org.apache.hadoop.fs.Path(f.toString()), overwrite);
+				new org.apache.hadoop.fs.Path(f.toString()), overwrite == WriteMode.OVERWRITE);
 
 		return new HadoopDataOutputStream(fdos);
 	}


[08/19] flink git commit: [FLINK-5739] [client] Fix NullPointerException in CliFrontend

Posted by se...@apache.org.
[FLINK-5739] [client] Fix NullPointerException in CliFrontend

This closes #3292


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

Branch: refs/heads/master
Commit: 5e32eb549d3bc2195548620005fcf54437e75f48
Parents: 3104619
Author: Zhuoluo Yang <zh...@alibaba-inc.com>
Authored: Tue Feb 14 09:54:50 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/flink/client/CliFrontend.java     | 6 ++++++
 .../flink/optimizer/plantranslate/JobGraphGenerator.java       | 3 ++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5e32eb54/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index b6543a3..8c84c5a 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -842,6 +842,12 @@ public class CliFrontend {
 			program.deleteExtractedLibraries();
 		}
 
+		if (null == result) {
+			logAndSysout("No JobSubmissionResult returned, please make sure you called " +
+				"ExecutionEnvironment.execute()");
+			return 1;
+		}
+
 		if (result.isJobExecutionResult()) {
 			logAndSysout("Program execution finished");
 			JobExecutionResult execResult = result.getJobExecutionResult();

http://git-wip-us.apache.org/repos/asf/flink/blob/5e32eb54/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
index 4ccfae3..6f7b04a 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
@@ -172,7 +172,8 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 	
 	public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) {
 		if (program == null) {
-			throw new NullPointerException();
+			throw new NullPointerException("Program is null, did you called " +
+				"ExecutionEnvironment.execute()");
 		}
 		
 		if (jobId == null) {


[12/19] flink git commit: [FLINK-5640] [build] Configure the explicit Unit Test file suffix

Posted by se...@apache.org.
[FLINK-5640] [build] Configure the explicit Unit Test file suffix

This closes #3211


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

Branch: refs/heads/master
Commit: 4ce2557da623d84f160fa3993b8e27590752fedb
Parents: 1456f0a
Author: shijinkui <sh...@huawei.com>
Authored: Fri Feb 17 00:24:20 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 pom.xml | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4ce2557d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5487d70..e6fae81 100644
--- a/pom.xml
+++ b/pom.xml
@@ -965,6 +965,8 @@ under the License.
 					<compilerArgument>-Xlint:all</compilerArgument>
 				</configuration>
 			</plugin>
+
+			<!--surefire for unit tests and integration tests-->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-surefire-plugin</artifactId>
@@ -980,6 +982,7 @@ under the License.
 					<argLine>-Xms256m -Xmx800m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
 				</configuration>
 				<executions>
+					<!--execute all the unit tests-->
 					<execution>
 						<id>default-test</id>
 						<phase>test</phase>
@@ -987,12 +990,16 @@ under the License.
 							<goal>test</goal>
 						</goals>
 						<configuration>
+							<includes>
+								<include>**/*Test.*</include>
+							</includes>
 							<excludes>
 								<exclude>**/*ITCase.*</exclude>
 								<exclude>${flink-fast-tests-pattern}</exclude>
 							</excludes>
 						</configuration>
 					</execution>
+					<!--execute all the integration tests-->
 					<execution>
 						<id>integration-tests</id>
 						<phase>integration-test</phase>


[18/19] flink git commit: [hotfix] [tests] Use random actor names in JobManagerHARecoveryTest to avoid name collisions

Posted by se...@apache.org.
[hotfix] [tests] Use random actor names in JobManagerHARecoveryTest to avoid name collisions


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

Branch: refs/heads/master
Commit: 31c26e3fc06ad1939284249547c3885a5b62a8f3
Parents: 082d40f
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Feb 17 20:01:25 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:24 2017 +0100

----------------------------------------------------------------------
 .../flink/runtime/jobmanager/JobManagerHARecoveryTest.java   | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/31c26e3f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
index 8985a34..5f2edac 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
@@ -169,9 +169,7 @@ public class JobManagerHARecoveryTest {
 			InstanceManager instanceManager = new InstanceManager();
 			instanceManager.addInstanceListener(scheduler);
 
-			archive = system.actorOf(Props.create(
-					MemoryArchivist.class,
-					10), "archive");
+			archive = system.actorOf(Props.create(MemoryArchivist.class, 10));
 
 			Props jobManagerProps = Props.create(
 				TestingJobManager.class,
@@ -190,7 +188,7 @@ public class JobManagerHARecoveryTest {
 				jobRecoveryTimeout,
 				Option.apply(null));
 
-			jobManager = system.actorOf(jobManagerProps, "jobmanager");
+			jobManager = system.actorOf(jobManagerProps);
 			ActorGateway gateway = new AkkaActorGateway(jobManager, leaderSessionID);
 
 			taskManager = TaskManager.startTaskManagerComponentsAndActor(
@@ -360,7 +358,7 @@ public class JobManagerHARecoveryTest {
 				Option.<MetricRegistry>apply(null),
 				recoveredJobs).withDispatcher(CallingThreadDispatcher.Id());
 
-			jobManager = system.actorOf(jobManagerProps, "jobmanager");
+			jobManager = system.actorOf(jobManagerProps);
 
 			Future<Object> started = Patterns.ask(jobManager, new Identify(42), deadline.timeLeft().toMillis());
 


[05/19] flink git commit: [hotfix] [docs] Updated DC/OS setup instructions.

Posted by se...@apache.org.
[hotfix] [docs] Updated DC/OS setup instructions.

This closes #3349


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

Branch: refs/heads/master
Commit: e68ee5cb18ee8ebd50d17604608993790271929c
Parents: 20420fc
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Fri Feb 17 16:10:53 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Feb 18 19:19:34 2017 +0100

----------------------------------------------------------------------
 docs/setup/mesos.md | 37 ++++++++++---------------------------
 1 file changed, 10 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e68ee5cb/docs/setup/mesos.md
----------------------------------------------------------------------
diff --git a/docs/setup/mesos.md b/docs/setup/mesos.md
index 77267cb..2ccee65 100644
--- a/docs/setup/mesos.md
+++ b/docs/setup/mesos.md
@@ -34,7 +34,7 @@ set up by the application master. The most sophisticated component of the Mesos
 implementation is the application master. The application master currently hosts
 the following components:
 
-### Mesos Scheduler 
+### Mesos Scheduler
 
 The scheduler is responsible for registering the framework with Mesos,
 requesting resources, and launching worker nodes. The scheduler continuously
@@ -57,7 +57,7 @@ The artifact server is responsible for providing resources to the worker
 nodes. The resources can be anything from the Flink binaries to shared secrets
 or configuration files. For instance, in non-containered environments, the
 artifact server will provide the Flink binaries. What files will be served
-depends on the configuration overlay used. 
+depends on the configuration overlay used.
 
 ### Flink's JobManager and Web Interface
 
@@ -87,30 +87,13 @@ If you don't have a running DC/OS cluster, please follow the
 [instructions on how to install DC/OS on the official website](https://dcos.io/install/).
 
 Once you have a DC/OS cluster, you may install Flink through the DC/OS
-Universe. In the search prompt, just search for Flink. 
+Universe. In the search prompt, just search for Flink. Alternatively, you can use the DC/OS CLI:
 
-**Note**: At the time of this writing, Flink was not yet available in the
-Universe. Please use the following workaround in the meantime:
+    dcos package install flink
 
-1. [Install the DC/OS CLI](https://dcos.io/docs/1.8/usage/cli/install/)
+Further information can be found in the
+[DC/OS examples documentation](https://github.com/dcos/examples/tree/master/1.8/flink).
 
-2. Add the Development Universe
-
-    `./dcos marathon app add https://raw.githubusercontent.com/mesosphere/dcos-flink-service/Makman2/quickstart/universe-server.json`
-    
-3. Add the local Universe repository:
-
-   `./dcos package repo add --index=0 dev-universe http://universe.marathon.mesos:8085/repo`
-
-4. Install Flink through the Universe page or using the `dcos` command:
-   
-   `./dcos package install flink`
-
-In order to execute a Flink job on a DC/OS hosted Flink cluster, you first have to find out the address of the launched JobManager.
-The JobManager address can be found out by opening the Flink service, going to *Job Manager* and then using the address specified under `jobmanager.rpc.address` and `jobmanager.rpc.port`.
-Now you can use this address to submit a job to your cluster via
-
-    FLINK_HOME/bin/flink run -m address:port flink-job.jar
 
 ## Mesos without DC/OS
 
@@ -167,7 +150,7 @@ A more convenient and easier to maintain approach is to use Docker containers to
 This is controlled via the following configuration entries:
 
     mesos.resourcemanager.tasks.container.type: mesos _or_ docker
-    
+
 If set to 'docker', specify the image name:
 
     mesos.resourcemanager.tasks.container.image.name: image_name
@@ -181,7 +164,7 @@ which manage the Flink processes in a Mesos cluster:
 1. `mesos-appmaster.sh`
    This starts the Mesos application master which will register the Mesos scheduler.
    It is also responsible for starting up the worker nodes.
-   
+
 2. `mesos-taskmanager.sh`
    The entry point for the Mesos worker processes.
    You don't need to explicitly execute this script.
@@ -241,14 +224,14 @@ When running Flink with Marathon, the whole Flink cluster including the job mana
 
 `mesos.maximum-failed-tasks`: The maximum number of failed workers before the cluster fails (**DEFAULT**: Number of initial workers).
 May be set to -1 to disable this feature.
-    
+
 `mesos.master`: The Mesos master URL. The value should be in one of the following forms:
 
 * `host:port`
 * `zk://host1:port1,host2:port2,.../path`
 * `zk://username:password@host1:port1,host2:port2,.../path`
 * `file:///path/to/file`
-     
+
 `mesos.failover-timeout`: The failover timeout in seconds for the Mesos scheduler, after which running tasks are automatically shut down (**DEFAULT:** 600).
 
 `mesos.resourcemanager.artifactserver.port`:The config parameter defining the Mesos artifact server port to use. Setting the port to 0 will let the OS choose an available port.


[07/19] flink git commit: [hotfix] [core] Add missing @PublicEvolving annotations to classes in flink-core.

Posted by se...@apache.org.
[hotfix] [core] Add missing @PublicEvolving annotations to classes in flink-core.


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

Branch: refs/heads/master
Commit: 0aa9918cc5b56682087c2c19eb31d9b321c97875
Parents: 391efd3
Author: Fabian Hueske <fh...@apache.org>
Authored: Wed Feb 15 15:25:01 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/flink/api/common/Archiveable.java  | 3 +++
 .../java/org/apache/flink/configuration/SecurityOptions.java    | 3 +++
 .../java/org/apache/flink/core/io/VersionMismatchException.java | 5 ++++-
 .../src/main/java/org/apache/flink/core/io/Versioned.java       | 3 +++
 .../org/apache/flink/core/io/VersionedIOReadableWritable.java   | 4 +++-
 .../apache/flink/migration/util/MigrationInstantiationUtil.java | 4 +++-
 .../java/org/apache/flink/migration/util/SerializedValue.java   | 2 ++
 7 files changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/api/common/Archiveable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Archiveable.java b/flink-core/src/main/java/org/apache/flink/api/common/Archiveable.java
index 09a3a0c..69e050d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/Archiveable.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/Archiveable.java
@@ -17,8 +17,11 @@
  */
 package org.apache.flink.api.common;
 
+import org.apache.flink.annotation.PublicEvolving;
+
 import java.io.Serializable;
 
+@PublicEvolving
 public interface Archiveable<T extends Serializable> {
 	T archive();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
index 67d101d..95cf0c7 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
@@ -18,11 +18,14 @@
 
 package org.apache.flink.configuration;
 
+import org.apache.flink.annotation.PublicEvolving;
+
 import static org.apache.flink.configuration.ConfigOptions.key;
 
 /**
  * The set of configuration options relating to security.
  */
+@PublicEvolving
 public class SecurityOptions {
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/core/io/VersionMismatchException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/VersionMismatchException.java b/flink-core/src/main/java/org/apache/flink/core/io/VersionMismatchException.java
index 3ff88e9..92fd4f4 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/VersionMismatchException.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/VersionMismatchException.java
@@ -18,11 +18,14 @@
 
 package org.apache.flink.core.io;
 
+import org.apache.flink.annotation.PublicEvolving;
+
 import java.io.IOException;
 
 /**
  * This exception signals that incompatible versions have been found during serialization.
  */
+@PublicEvolving
 public class VersionMismatchException extends IOException {
 
 	private static final long serialVersionUID = 7024258967585372438L;
@@ -41,4 +44,4 @@ public class VersionMismatchException extends IOException {
 	public VersionMismatchException(Throwable cause) {
 		super(cause);
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/core/io/Versioned.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/Versioned.java b/flink-core/src/main/java/org/apache/flink/core/io/Versioned.java
index b36d5e8..786bf73 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/Versioned.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/Versioned.java
@@ -18,10 +18,13 @@
 
 package org.apache.flink.core.io;
 
+import org.apache.flink.annotation.PublicEvolving;
+
 /**
  * This interface is implemented by classes that provide a version number. Versions numbers can be used to differentiate
  * between evolving classes.
  */
+@PublicEvolving
 public interface Versioned {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/core/io/VersionedIOReadableWritable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/VersionedIOReadableWritable.java b/flink-core/src/main/java/org/apache/flink/core/io/VersionedIOReadableWritable.java
index 94c2722..31d570c 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/VersionedIOReadableWritable.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/VersionedIOReadableWritable.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.io;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
@@ -28,6 +29,7 @@ import java.io.IOException;
  * versions. Concrete subclasses should typically override the {@link #write(DataOutputView)} and
  * {@link #read(DataInputView)}, thereby calling super to ensure version checking.
  */
+@PublicEvolving
 public abstract class VersionedIOReadableWritable implements IOReadableWritable, Versioned {
 
 	@Override
@@ -67,4 +69,4 @@ public abstract class VersionedIOReadableWritable implements IOReadableWritable,
 	public boolean isCompatibleVersion(int version) {
 		return getVersion() == version;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
index d175b2f..69e4e6d 100644
--- a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.migration.util;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.util.InstantiationUtil;
 
 import java.io.ByteArrayInputStream;
@@ -29,6 +30,7 @@ import java.io.ObjectStreamClass;
 /**
  * Utility class to deserialize legacy classes for migration.
  */
+@PublicEvolving
 public final class MigrationInstantiationUtil {
 
 	public static class ClassLoaderObjectInputStream extends InstantiationUtil.ClassLoaderObjectInputStream {
@@ -91,4 +93,4 @@ public final class MigrationInstantiationUtil {
 		throw new IllegalAccessError();
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0aa9918c/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java b/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
index aab68c9..bd40b83 100644
--- a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
+++ b/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.migration.util;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.util.InstantiationUtil;
 
 import java.io.IOException;
@@ -36,6 +37,7 @@ import java.util.Arrays;
  * @param <T> The type of the value held.
  */
 @Deprecated
+@PublicEvolving
 public class SerializedValue<T> implements java.io.Serializable {
 
 	private static final long serialVersionUID = -3564011643393683761L;


[16/19] flink git commit: [FLINK-5747] [distributed coordination] Eager scheduling allocates slots and deploys tasks in bulk

Posted by se...@apache.org.
[FLINK-5747] [distributed coordination] Eager scheduling allocates slots and deploys tasks in bulk

That way, strictly topological deployment can be guaranteed.

Also, many quick deploy/not-enough-resources/fail/recover cycles can be
avoided in the cases where resources need some time to appear.

This closes #3295


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

Branch: refs/heads/master
Commit: f113d79451ba88c487358861cc3e20aac3d19257
Parents: 5902ea0
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Feb 3 20:26:23 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:24 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |  12 +
 .../apache/flink/util/ExceptionUtilsTest.java   |  60 ++
 .../flink/runtime/concurrent/Executors.java     |   3 +-
 .../flink/runtime/concurrent/FutureUtils.java   | 115 ++++
 .../flink/runtime/executiongraph/Execution.java |  73 ++-
 .../executiongraph/ExecutionAndSlot.java        |  46 ++
 .../runtime/executiongraph/ExecutionGraph.java  | 170 +++++-
 .../executiongraph/ExecutionGraphUtils.java     | 106 ++++
 .../executiongraph/ExecutionJobVertex.java      |  46 +-
 .../runtime/executiongraph/ExecutionVertex.java |   3 +-
 .../IllegalExecutionStateException.java         |  53 ++
 .../apache/flink/runtime/instance/SlotPool.java |   9 +-
 .../runtime/concurrent/FutureUtilsTest.java     | 194 ++++++
 .../ExecutionGraphSchedulingTest.java           | 610 +++++++++++++++++++
 .../executiongraph/ExecutionGraphUtilsTest.java | 124 ++++
 .../ExecutionVertexCancelTest.java              |   2 +-
 .../ExecutionVertexSchedulingTest.java          |   3 -
 .../executiongraph/PointwisePatternTest.java    |  12 +-
 .../executiongraph/ProgrammedSlotProvider.java  |  87 +++
 .../TerminalJobStatusListener.java              |  45 ++
 .../LeaderChangeJobRecoveryTest.java            |  23 +-
 .../runtime/minicluster/MiniClusterITCase.java  |  28 +-
 .../Flip6LocalStreamEnvironment.java            |   4 +-
 23 files changed, 1735 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
index 6ba9ef6..69c2692 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -101,6 +101,18 @@ public final class ExceptionUtils {
 	}
 
 	/**
+	 * Rethrows the given {@code Throwable}, if it represents an error that is fatal to the JVM.
+	 * See {@link ExceptionUtils#isJvmFatalError(Throwable)} for a definition of fatal errors.
+	 * 
+	 * @param t The Throwable to check and rethrow.
+	 */
+	public static void rethrowIfFatalError(Throwable t) {
+		if (isJvmFatalError(t)) {
+			throw (Error) t;
+		}
+	}
+
+	/**
 	 * Adds a new exception as a {@link Throwable#addSuppressed(Throwable) suppressed exception}
 	 * to a prior exception, or returns the new exception, if no prior exception exists.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
new file mode 100644
index 0000000..343b9d6
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.util;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the utility methods in {@link ExceptionUtils}.
+ */
+public class ExceptionUtilsTest {
+
+	@Test
+	public void testStringifyNullException() {
+		assertNotNull(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION);
+		assertEquals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION, ExceptionUtils.stringifyException(null));
+	}
+
+	@Test
+	public void testJvmFatalError() {
+		// not all errors are fatal
+		assertFalse(ExceptionUtils.isJvmFatalError(new Error()));
+
+		// linkage errors are not fatal
+		assertFalse(ExceptionUtils.isJvmFatalError(new LinkageError()));
+
+		// some errors are fatal
+		assertTrue(ExceptionUtils.isJvmFatalError(new InternalError()));
+		assertTrue(ExceptionUtils.isJvmFatalError(new UnknownError()));
+	}
+
+	@Test
+	public void testRethrowFatalError() {
+		// fatal error is rethrown
+		try {
+			ExceptionUtils.rethrowIfFatalError(new InternalError());
+			fail();
+		} catch (InternalError ignored) {}
+
+		// non-fatal error is not rethrown
+		ExceptionUtils.rethrowIfFatalError(new NoClassDefFoundError());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java
index 391f233..63b6a25 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.concurrent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -52,7 +53,7 @@ public class Executors {
 		private DirectExecutor() {}
 
 		@Override
-		public void execute(Runnable command) {
+		public void execute(@Nonnull Runnable command) {
 			command.run();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
index a404c98..4948147 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
@@ -20,11 +20,22 @@ package org.apache.flink.runtime.concurrent;
 
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 
+import java.util.Collection;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A collection of utilities that expand the usage of {@link Future} and {@link CompletableFuture}.
+ */
 public class FutureUtils {
 
+	// ------------------------------------------------------------------------
+	//  retrying operations
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Retry the given operation the given number of times in case of a failure.
 	 *
@@ -88,4 +99,108 @@ public class FutureUtils {
 			super(cause);
 		}
 	}
+
+	// ------------------------------------------------------------------------
+	//  composing futures
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a future that is complete once multiple other futures completed. 
+	 * The ConjunctFuture fails (completes exceptionally) once one of the Futures in the
+	 * conjunction fails.
+	 *
+	 * <p>The ConjunctFuture gives access to how many Futures in the conjunction have already
+	 * completed successfully, via {@link ConjunctFuture#getNumFuturesCompleted()}. 
+	 * 
+	 * @param futures The futures that make up the conjunction. No null entries are allowed.
+	 * @return The ConjunctFuture that completes once all given futures are complete (or one fails).
+	 */
+	public static ConjunctFuture combineAll(Collection<? extends Future<?>> futures) {
+		checkNotNull(futures, "futures");
+
+		final ConjunctFutureImpl conjunct = new ConjunctFutureImpl(futures.size());
+
+		if (futures.isEmpty()) {
+			conjunct.complete(null);
+		}
+		else {
+			for (Future<?> future : futures) {
+				future.handle(conjunct.completionHandler);
+			}
+		}
+
+		return conjunct;
+	}
+
+	/**
+	 * A future that is complete once multiple other futures completed. The futures are not
+	 * necessarily of the same type, which is why the type of this Future is {@code Void}.
+	 * The ConjunctFuture fails (completes exceptionally) once one of the Futures in the
+	 * conjunction fails.
+	 * 
+	 * <p>The advantage of using the ConjunctFuture over chaining all the futures (such as via
+	 * {@link Future#thenCombine(Future, BiFunction)}) is that ConjunctFuture also tracks how
+	 * many of the Futures are already complete.
+	 */
+	public interface ConjunctFuture extends CompletableFuture<Void> {
+
+		/**
+		 * Gets the total number of Futures in the conjunction.
+		 * @return The total number of Futures in the conjunction.
+		 */
+		int getNumFuturesTotal();
+
+		/**
+		 * Gets the number of Futures in the conjunction that are already complete.
+		 * @return The number of Futures in the conjunction that are already complete
+		 */
+		int getNumFuturesCompleted();
+	}
+
+	/**
+	 * The implementation of the {@link ConjunctFuture}.
+	 * 
+	 * <p>Implementation notice: The member fields all have package-private access, because they are
+	 * either accessed by an inner subclass or by the enclosing class.
+	 */
+	private static class ConjunctFutureImpl extends FlinkCompletableFuture<Void> implements ConjunctFuture {
+
+		/** The total number of futures in the conjunction */
+		final int numTotal;
+
+		/** The number of futures in the conjunction that are already complete */
+		final AtomicInteger numCompleted = new AtomicInteger();
+
+		/** The function that is attached to all futures in the conjunction. Once a future
+		 * is complete, this function tracks the completion or fails the conjunct.  
+		 */
+		final BiFunction<Object, Throwable, Void> completionHandler = new BiFunction<Object, Throwable, Void>() {
+
+			@Override
+			public Void apply(Object o, Throwable throwable) {
+				if (throwable != null) {
+					completeExceptionally(throwable);
+				}
+				else if (numTotal == numCompleted.incrementAndGet()) {
+					complete(null);
+				}
+
+				return null;
+			}
+		};
+
+		ConjunctFutureImpl(int numTotal) {
+			this.numTotal = numTotal;
+		}
+
+		@Override
+		public int getNumFuturesTotal() {
+			return numTotal;
+		}
+
+		@Override
+		public int getNumFuturesCompleted() {
+			return numCompleted.get();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 60e5575..b3fe443 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -249,27 +249,8 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
 	 */
 	public boolean scheduleForExecution(SlotProvider slotProvider, boolean queued) {
-		if (slotProvider == null) {
-			throw new IllegalArgumentException("Cannot send null Scheduler when scheduling execution.");
-		}
-
-		final SlotSharingGroup sharingGroup = vertex.getJobVertex().getSlotSharingGroup();
-		final CoLocationConstraint locationConstraint = vertex.getLocationConstraint();
-
-		// sanity check
-		if (locationConstraint != null && sharingGroup == null) {
-			throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing allowed.");
-		}
-
-		if (transitionState(CREATED, SCHEDULED)) {
-
-			ScheduledUnit toSchedule = locationConstraint == null ?
-				new ScheduledUnit(this, sharingGroup) :
-				new ScheduledUnit(this, sharingGroup, locationConstraint);
-
-			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
-			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
-			final Future<SimpleSlot> slotAllocationFuture = slotProvider.allocateSlot(toSchedule, queued);
+		try {
+			final Future<SimpleSlot> slotAllocationFuture = allocateSlotForExecution(slotProvider, queued);
 
 			// IMPORTANT: We have to use the synchronous handle operation (direct executor) here so
 			// that we directly deploy the tasks if the slot allocation future is completed. This is
@@ -296,28 +277,54 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 			});
 
 			// if tasks have to scheduled immediately check that the task has been deployed
-			// TODO: This might be problematic if the future is not completed right away
-			if (!queued) {
-				if (!deploymentFuture.isDone()) {
-					markFailed(new IllegalArgumentException("The slot allocation future has not been completed yet."));
-				}
+			if (!queued && !deploymentFuture.isDone()) {
+				markFailed(new IllegalArgumentException("The slot allocation future has not been completed yet."));
 			}
-			
+
 			return true;
 		}
+		catch (IllegalExecutionStateException e) {
+			return false;
+		}
+	}
+
+	public Future<SimpleSlot> allocateSlotForExecution(SlotProvider slotProvider, boolean queued) 
+			throws IllegalExecutionStateException {
+
+		checkNotNull(slotProvider);
+
+		final SlotSharingGroup sharingGroup = vertex.getJobVertex().getSlotSharingGroup();
+		final CoLocationConstraint locationConstraint = vertex.getLocationConstraint();
+
+		// sanity check
+		if (locationConstraint != null && sharingGroup == null) {
+			throw new IllegalStateException(
+					"Trying to schedule with co-location constraint but without slot sharing allowed.");
+		}
+
+		// this method only works if the execution is in the state 'CREATED'
+		if (transitionState(CREATED, SCHEDULED)) {
+
+			ScheduledUnit toSchedule = locationConstraint == null ?
+					new ScheduledUnit(this, sharingGroup) :
+					new ScheduledUnit(this, sharingGroup, locationConstraint);
+
+			return slotProvider.allocateSlot(toSchedule, queued);
+		}
 		else {
 			// call race, already deployed, or already done
-			return false;
+			throw new IllegalExecutionStateException(this, CREATED, state);
 		}
 	}
 
 	public void deployToSlot(final SimpleSlot slot) throws JobException {
-		// sanity checks
-		if (slot == null) {
-			throw new NullPointerException();
-		}
+		checkNotNull(slot);
+
+		// Check if the TaskManager died in the meantime
+		// This only speeds up the response to TaskManagers failing concurrently to deployments.
+		// The more general check is the timeout of the deployment call
 		if (!slot.isAlive()) {
-			throw new JobException("Target slot for deployment is not alive.");
+			throw new JobException("Target slot (TaskManager) for deployment is no longer alive.");
 		}
 
 		// make sure exactly one deployment call happens from the correct state

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAndSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAndSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAndSlot.java
new file mode 100644
index 0000000..ea6186e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAndSlot.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.instance.SimpleSlot;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A pair of an {@link Execution} together with a slot future.
+ */
+public class ExecutionAndSlot {
+
+	public final Execution executionAttempt;
+
+	public final Future<SimpleSlot> slotFuture;
+
+	public ExecutionAndSlot(Execution executionAttempt, Future<SimpleSlot> slotFuture) {
+		this.executionAttempt = checkNotNull(executionAttempt);
+		this.slotFuture = checkNotNull(slotFuture);
+	}
+
+	// -----------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return super.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index f25120c..ad4347d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.executiongraph;
 
 import org.apache.commons.lang3.StringUtils;
+
 import org.apache.flink.api.common.Archiveable;
 import org.apache.flink.api.common.ArchivedExecutionConfig;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -40,9 +41,14 @@ import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -53,6 +59,7 @@ import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.query.KvStateLocationRegistry;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializableObject;
@@ -60,6 +67,7 @@ import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.SerializedValue;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -77,11 +85,14 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 /**
  * The execution graph is the central data structure that coordinates the distributed
@@ -158,7 +169,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	private final long[] stateTimestamps;
 
 	/** The timeout for all messages that require a response/acknowledgement */
-	private final Time timeout;
+	private final Time rpcCallTimeout;
 
 	// ------ Configuration of the Execution -------
 
@@ -171,6 +182,8 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	 * from results than need to be materialized. */
 	private ScheduleMode scheduleMode = ScheduleMode.LAZY_FROM_SOURCES;
 
+	private final Time scheduleAllocationTimeout;
+
 	// ------ Execution status and progress. These values are volatile, and accessed under the lock -------
 
 	/** Current status of the job execution */
@@ -292,7 +305,8 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		this.stateTimestamps = new long[JobStatus.values().length];
 		this.stateTimestamps[JobStatus.CREATED.ordinal()] = System.currentTimeMillis();
 
-		this.timeout = timeout;
+		this.rpcCallTimeout = checkNotNull(timeout);
+		this.scheduleAllocationTimeout = checkNotNull(timeout);
 
 		this.restartStrategy = restartStrategy;
 
@@ -695,7 +709,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 			// create the execution job vertex and attach it to the graph
 			ExecutionJobVertex ejv =
-					new ExecutionJobVertex(this, jobVertex, 1, timeout, createTimestamp);
+					new ExecutionJobVertex(this, jobVertex, 1, rpcCallTimeout, createTimestamp);
 			ejv.connectToPredecessors(this.intermediateResults);
 
 			ExecutionJobVertex previousTask = this.tasks.putIfAbsent(jobVertex.getID(), ejv);
@@ -717,9 +731,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	}
 
 	public void scheduleForExecution(SlotProvider slotProvider) throws JobException {
-		if (slotProvider == null) {
-			throw new IllegalArgumentException("Scheduler must not be null.");
-		}
+		checkNotNull(slotProvider);
 
 		if (this.slotProvider != null && this.slotProvider != slotProvider) {
 			throw new IllegalArgumentException("Cannot use different slot providers for the same job");
@@ -731,18 +743,11 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			switch (scheduleMode) {
 
 				case LAZY_FROM_SOURCES:
-					// simply take the vertices without inputs.
-					for (ExecutionJobVertex ejv : this.tasks.values()) {
-						if (ejv.getJobVertex().isInputVertex()) {
-							ejv.scheduleAll(slotProvider, allowQueuedScheduling);
-						}
-					}
+					scheduleLazy(slotProvider);
 					break;
 
 				case EAGER:
-					for (ExecutionJobVertex ejv : getVerticesTopologically()) {
-						ejv.scheduleAll(slotProvider, allowQueuedScheduling);
-					}
+					scheduleEager(slotProvider, scheduleAllocationTimeout);
 					break;
 
 				default:
@@ -754,6 +759,139 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		}
 	}
 
+	private void scheduleLazy(SlotProvider slotProvider) throws NoResourceAvailableException {
+		// simply take the vertices without inputs.
+		for (ExecutionJobVertex ejv : this.tasks.values()) {
+			if (ejv.getJobVertex().isInputVertex()) {
+				ejv.scheduleAll(slotProvider, allowQueuedScheduling);
+			}
+		}
+	}
+
+	/**
+	 * 
+	 * 
+	 * @param slotProvider  The resource provider from which the slots are allocated
+	 * @param timeout       The maximum time that the deployment may take, before a
+	 *                      TimeoutException is thrown.
+	 */
+	private void scheduleEager(SlotProvider slotProvider, final Time timeout) {
+		checkState(state == JobStatus.RUNNING, "job is not running currently");
+
+		// Important: reserve all the space we need up front.
+		// that way we do not have any operation that can fail between allocating the slots
+		// and adding them to the list. If we had a failure in between there, that would
+		// cause the slots to get lost
+		final ArrayList<ExecutionAndSlot[]> resources = new ArrayList<>(getNumberOfExecutionJobVertices());
+		final boolean queued = allowQueuedScheduling;
+
+		// we use this flag to handle failures in a 'finally' clause
+		// that allows us to not go through clumsy cast-and-rethrow logic
+		boolean successful = false;
+
+		try {
+			// collecting all the slots may resize and fail in that operation without slots getting lost
+			final ArrayList<Future<SimpleSlot>> slotFutures = new ArrayList<>(getNumberOfExecutionJobVertices());
+
+			// allocate the slots (obtain all their futures
+			for (ExecutionJobVertex ejv : getVerticesTopologically()) {
+				// these calls are not blocking, they only return futures
+				ExecutionAndSlot[] slots = ejv.allocateResourcesForAll(slotProvider, queued);
+
+				// we need to first add the slots to this list, to be safe on release
+				resources.add(slots);
+
+				for (ExecutionAndSlot ens : slots) {
+					slotFutures.add(ens.slotFuture);
+				}
+			}
+
+			// this future is complete once all slot futures are complete.
+			// the future fails once one slot future fails.
+			final ConjunctFuture allAllocationsComplete = FutureUtils.combineAll(slotFutures);
+
+			// make sure that we fail if the allocation timeout was exceeded
+			final ScheduledFuture<?> timeoutCancelHandle = futureExecutor.schedule(new Runnable() {
+				@Override
+				public void run() {
+					// When the timeout triggers, we try to complete the conjunct future with an exception.
+					// Note that this is a no-op if the future is already completed
+					int numTotal = allAllocationsComplete.getNumFuturesTotal();
+					int numComplete = allAllocationsComplete.getNumFuturesCompleted();
+					String message = "Could not allocate all requires slots within timeout of " +
+							timeout + ". Slots required: " + numTotal + ", slots allocated: " + numComplete;
+
+					allAllocationsComplete.completeExceptionally(new NoResourceAvailableException(message));
+				}
+			}, timeout.getSize(), timeout.getUnit());
+
+
+			allAllocationsComplete.handleAsync(new BiFunction<Void, Throwable, Void>() {
+
+				@Override
+				public Void apply(Void ignored, Throwable throwable) {
+					try {
+						// we do not need the cancellation timeout any more
+						timeoutCancelHandle.cancel(false);
+
+						if (throwable == null) {
+							// successfully obtained all slots, now deploy
+
+							for (ExecutionAndSlot[] jobVertexTasks : resources) {
+								for (ExecutionAndSlot execAndSlot : jobVertexTasks) {
+
+									// the futures must all be ready - this is simply a sanity check
+									final SimpleSlot slot;
+									try {
+										slot = execAndSlot.slotFuture.getNow(null);
+										checkNotNull(slot);
+									}
+									catch (ExecutionException | NullPointerException e) {
+										throw new IllegalStateException("SlotFuture is incomplete " +
+												"or erroneous even though all futures completed");
+									}
+
+									// actual deployment
+									execAndSlot.executionAttempt.deployToSlot(slot);
+								}
+							}
+						}
+						else {
+							// let the exception handler deal with this
+							throw throwable;
+						}
+					}
+					catch (Throwable t) {
+						// we catch everything here to make sure cleanup happens and the
+						// ExecutionGraph notices
+						// we need to go into recovery and make sure to release all slots
+						try {
+							fail(t);
+						}
+						finally {
+							ExecutionGraphUtils.releaseAllSlotsSilently(resources);
+						}
+					}
+
+					// Wouldn't it be nice if we could return an actual Void object?
+					// return (Void) Unsafe.getUnsafe().allocateInstance(Void.class);
+					return null; 
+				}
+			}, futureExecutor);
+
+			// from now on, slots will be rescued by the the futures and their completion, or by the timeout
+			successful = true;
+		}
+		finally {
+			if (!successful) {
+				// we come here only if the 'try' block finished with an exception
+				// we release the slots (possibly failing some executions on the way) and
+				// let the exception bubble up
+				ExecutionGraphUtils.releaseAllSlotsSilently(resources);
+			}
+		}
+	}
+
 	public void cancel() {
 		while (true) {
 			JobStatus current = state;
@@ -971,7 +1109,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			}
 		} catch (IOException | ClassNotFoundException e) {
 			LOG.error("Couldn't create ArchivedExecutionConfig for job {} ", getJobID(), e);
-		};
+		}
 		return null;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtils.java
new file mode 100644
index 0000000..cd6d6aa
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtils.java
@@ -0,0 +1,106 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.List;
+
+/**
+ * Utilities for dealing with the execution graphs and scheduling.
+ */
+public class ExecutionGraphUtils {
+
+	/**
+	 * Releases the slot represented by the given future. If the future is complete, the
+	 * slot is immediately released. Otherwise, the slot is released as soon as the future
+	 * is completed.
+	 * 
+	 * <p>Note that releasing the slot means cancelling any task execution currently
+	 * associated with that slot.
+	 * 
+	 * @param slotFuture The future for the slot to release.
+	 */
+	public static void releaseSlotFuture(Future<SimpleSlot> slotFuture) {
+		slotFuture.handle(ReleaseSlotFunction.INSTANCE);
+	}
+
+	/**
+	 * Releases the all the slots in the list of arrays of {@code ExecutionAndSlot}.
+	 * For each future in that collection holds: If the future is complete, its slot is
+	 * immediately released. Otherwise, the slot is released as soon as the future
+	 * is completed.
+	 * 
+	 * <p>This methods never throws any exceptions (except for fatal exceptions) and continues
+	 * to release the remaining slots if one slot release failed.
+	 *
+	 * <p>Note that releasing the slot means cancelling any task execution currently
+	 * associated with that slot.
+	 * 
+	 * @param resources The collection of ExecutionAndSlot whose slots should be released.
+	 */
+	public static void releaseAllSlotsSilently(List<ExecutionAndSlot[]> resources) {
+		try {
+			for (ExecutionAndSlot[] jobVertexResources : resources) {
+				if (jobVertexResources != null) {
+					for (ExecutionAndSlot execAndSlot : jobVertexResources) {
+						if (execAndSlot != null) {
+							try {
+								releaseSlotFuture(execAndSlot.slotFuture);
+							}
+							catch (Throwable t) {
+								ExceptionUtils.rethrowIfFatalError(t);
+							}
+						}
+					}
+				}
+			}
+		}
+		catch (Throwable t) {
+			ExceptionUtils.rethrowIfFatalError(t);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A function to be applied into a future, releasing the slot immediately upon completion.
+	 * Completion here refers to both the successful and exceptional completion.
+	 */
+	private static final class ReleaseSlotFunction implements BiFunction<SimpleSlot, Throwable, Void> {
+
+		static final ReleaseSlotFunction INSTANCE = new ReleaseSlotFunction();
+
+		@Override
+		public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
+			if (simpleSlot != null) {
+				simpleSlot.releaseSlot();
+			}
+			return null;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** Utility class is not meant to be instantiated */
+	private ExecutionGraphUtils() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index 3828fc9..754148e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -30,7 +30,9 @@ import org.apache.flink.core.io.InputSplitSource;
 import org.apache.flink.core.io.LocatableInputSplit;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -388,7 +390,7 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable
 	
 	public void scheduleAll(SlotProvider slotProvider, boolean queued) {
 		
-		ExecutionVertex[] vertices = this.taskVertices;
+		final ExecutionVertex[] vertices = this.taskVertices;
 
 		// kick off the tasks
 		for (ExecutionVertex ev : vertices) {
@@ -396,6 +398,48 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable
 		}
 	}
 
+	/**
+	 * Acquires a slot for all the execution vertices of this ExecutionJobVertex. The method returns
+	 * pairs of the slots and execution attempts, to ease correlation between vertices and execution
+	 * attempts.
+	 * 
+	 * <p>If this method throws an exception, it makes sure to release all so far requested slots.
+	 * 
+	 * @param resourceProvider The resource provider from whom the slots are requested.
+	 */
+	public ExecutionAndSlot[] allocateResourcesForAll(SlotProvider resourceProvider, boolean queued) {
+		final ExecutionVertex[] vertices = this.taskVertices;
+		final ExecutionAndSlot[] slots = new ExecutionAndSlot[vertices.length];
+
+		// try to acquire a slot future for each execution.
+		// we store the execution with the future just to be on the safe side
+		for (int i = 0; i < vertices.length; i++) {
+
+			// we use this flag to handle failures in a 'finally' clause
+			// that allows us to not go through clumsy cast-and-rethrow logic
+			boolean successful = false;
+
+			try {
+				// allocate the next slot (future)
+				final Execution exec = vertices[i].getCurrentExecutionAttempt();
+				final Future<SimpleSlot> future = exec.allocateSlotForExecution(resourceProvider, queued);
+				slots[i] = new ExecutionAndSlot(exec, future);
+				successful = true;
+			}
+			finally {
+				if (!successful) {
+					// this is the case if an exception was thrown
+					for (int k = 0; k < i; k++) {
+						ExecutionGraphUtils.releaseSlotFuture(slots[k].slotFuture);
+					}
+				}
+			}
+		}
+
+		// all good, we acquired all slots
+		return slots;
+	}
+
 	public void cancel() {
 		for (ExecutionVertex ev : getTaskVertices()) {
 			ev.cancel();

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 92327fd..ca8e07c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -102,6 +102,7 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 			int subTaskIndex,
 			IntermediateResult[] producedDataSets,
 			Time timeout) {
+
 		this(
 				jobVertex,
 				subTaskIndex,
@@ -133,7 +134,7 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 		this.taskNameWithSubtask = String.format("%s (%d/%d)",
 				jobVertex.getJobVertex().getName(), subTaskIndex + 1, jobVertex.getParallelism());
 
-		this.resultPartitions = new LinkedHashMap<IntermediateResultPartitionID, IntermediateResultPartition>(producedDataSets.length, 1);
+		this.resultPartitions = new LinkedHashMap<>(producedDataSets.length, 1);
 
 		for (IntermediateResult result : producedDataSets) {
 			IntermediateResultPartition irp = new IntermediateResultPartition(result, this, subTaskIndex);

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IllegalExecutionStateException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IllegalExecutionStateException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IllegalExecutionStateException.java
new file mode 100644
index 0000000..44162ab
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IllegalExecutionStateException.java
@@ -0,0 +1,53 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.runtime.execution.ExecutionState;
+
+/**
+ * A special {@link IllegalStateException} indicating a mismatch in the expected and actual
+ * {@link ExecutionState} of an {@link Execution}.
+ */
+public class IllegalExecutionStateException extends IllegalStateException {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates a new IllegalExecutionStateException with the error message indicating
+	 * the expected and actual state.
+	 * 
+	 * @param expected The expected state 
+	 * @param actual   The actual state
+	 */
+	public IllegalExecutionStateException(ExecutionState expected, ExecutionState actual) {
+		super("Invalid execution state: Expected " + expected + " , found " + actual);
+	}
+
+	/**
+	 * Creates a new IllegalExecutionStateException with the error message indicating
+	 * the expected and actual state.
+	 *
+	 * @param expected The expected state 
+	 * @param actual   The actual state
+	 */
+	public IllegalExecutionStateException(Execution execution, ExecutionState expected, ExecutionState actual) {
+		super(execution.getVertexWithAttempt() + " is no longer in expected state " + expected + 
+				" but in state " + actual);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
index 4da6c7b..8ba5040 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
@@ -1048,9 +1048,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 
 		private final long timestamp;
 
-		SlotAndTimestamp(
-				AllocatedSlot slot,
-				long timestamp) {
+		SlotAndTimestamp(AllocatedSlot slot, long timestamp) {
 			this.slot = slot;
 			this.timestamp = timestamp;
 		}
@@ -1062,5 +1060,10 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 		public long timestamp() {
 			return timestamp;
 		}
+
+		@Override
+		public String toString() {
+			return slot + " @ " + timestamp;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
new file mode 100644
index 0000000..43710cb
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.concurrent;
+
+import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the utility methods in {@link FutureUtils}
+ */
+public class FutureUtilsTest {
+
+	@Test
+	public void testConjunctFutureFailsOnEmptyAndNull() throws Exception {
+		try {
+			FutureUtils.combineAll(null);
+			fail();
+		} catch (NullPointerException ignored) {}
+
+		try {
+			FutureUtils.combineAll(Arrays.asList(
+					new FlinkCompletableFuture<Object>(),
+					null,
+					new FlinkCompletableFuture<Object>()));
+			fail();
+		} catch (NullPointerException ignored) {}
+	}
+
+	@Test
+	public void testConjunctFutureCompletion() throws Exception {
+		// some futures that we combine
+		CompletableFuture<Object> future1 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future2 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future3 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future4 = new FlinkCompletableFuture<>();
+
+		// some future is initially completed
+		future2.complete(new Object());
+
+		// build the conjunct future
+		ConjunctFuture result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3, future4));
+
+		Future<Void> resultMapped = result.thenAccept(new AcceptFunction<Void>() {
+			@Override
+			public void accept(Void value) {}
+		});
+
+		assertEquals(4, result.getNumFuturesTotal());
+		assertEquals(1, result.getNumFuturesCompleted());
+		assertFalse(result.isDone());
+		assertFalse(resultMapped.isDone());
+
+		// complete two more futures
+		future4.complete(new Object());
+		assertEquals(2, result.getNumFuturesCompleted());
+		assertFalse(result.isDone());
+		assertFalse(resultMapped.isDone());
+
+		future1.complete(new Object());
+		assertEquals(3, result.getNumFuturesCompleted());
+		assertFalse(result.isDone());
+		assertFalse(resultMapped.isDone());
+
+		// complete one future again
+		future1.complete(new Object());
+		assertEquals(3, result.getNumFuturesCompleted());
+		assertFalse(result.isDone());
+		assertFalse(resultMapped.isDone());
+
+		// complete the final future
+		future3.complete(new Object());
+		assertEquals(4, result.getNumFuturesCompleted());
+		assertTrue(result.isDone());
+		assertTrue(resultMapped.isDone());
+	}
+
+	@Test
+	public void testConjunctFutureFailureOnFirst() throws Exception {
+
+		CompletableFuture<Object> future1 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future2 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future3 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future4 = new FlinkCompletableFuture<>();
+
+		// build the conjunct future
+		ConjunctFuture result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3, future4));
+
+		Future<Void> resultMapped = result.thenAccept(new AcceptFunction<Void>() {
+			@Override
+			public void accept(Void value) {}
+		});
+
+		assertEquals(4, result.getNumFuturesTotal());
+		assertEquals(0, result.getNumFuturesCompleted());
+		assertFalse(result.isDone());
+		assertFalse(resultMapped.isDone());
+
+		future2.completeExceptionally(new IOException());
+
+		assertEquals(0, result.getNumFuturesCompleted());
+		assertTrue(result.isDone());
+		assertTrue(resultMapped.isDone());
+
+		try {
+			result.get();
+			fail();
+		} catch (ExecutionException e) {
+			assertTrue(e.getCause() instanceof IOException);
+		}
+
+		try {
+			resultMapped.get();
+			fail();
+		} catch (ExecutionException e) {
+			assertTrue(e.getCause() instanceof IOException);
+		}
+	}
+
+	@Test
+	public void testConjunctFutureFailureOnSuccessive() throws Exception {
+
+		CompletableFuture<Object> future1 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future2 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future3 = new FlinkCompletableFuture<>();
+		CompletableFuture<Object> future4 = new FlinkCompletableFuture<>();
+
+		// build the conjunct future
+		ConjunctFuture result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3, future4));
+		assertEquals(4, result.getNumFuturesTotal());
+
+		Future<Void> resultMapped = result.thenAccept(new AcceptFunction<Void>() {
+			@Override
+			public void accept(Void value) {}
+		});
+
+		future1.complete(new Object());
+		future3.complete(new Object());
+		future4.complete(new Object());
+
+		future2.completeExceptionally(new IOException());
+
+		assertEquals(3, result.getNumFuturesCompleted());
+		assertTrue(result.isDone());
+		assertTrue(resultMapped.isDone());
+
+		try {
+			result.get();
+			fail();
+		} catch (ExecutionException e) {
+			assertTrue(e.getCause() instanceof IOException);
+		}
+
+		try {
+			resultMapped.get();
+			fail();
+		} catch (ExecutionException e) {
+			assertTrue(e.getCause() instanceof IOException);
+		}
+	}
+
+	@Test
+	public void testConjunctOfNone() throws Exception {
+		final ConjunctFuture result = FutureUtils.combineAll(Collections.<Future<Object>>emptyList());
+
+		assertEquals(0, result.getNumFuturesTotal());
+		assertEquals(0, result.getNumFuturesCompleted());
+		assertTrue(result.isDone());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
new file mode 100644
index 0000000..9834dc6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
@@ -0,0 +1,610 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.Slot;
+import org.apache.flink.runtime.instance.SlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.SlotOwner;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.mockito.verification.Timeout;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the scheduling of the execution graph. This tests that
+ * for example the order of deployments is correct and that bulk slot allocation
+ * works properly.
+ */
+public class ExecutionGraphSchedulingTest extends TestLogger {
+
+	private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+
+	@After
+	public void shutdown() {
+		executor.shutdownNow();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Tests
+	// ------------------------------------------------------------------------
+
+	
+	/**
+	 * Tests that with scheduling futures and pipelined deployment, the target vertex will
+	 * not deploy its task before the source vertex does.
+	 */
+	@Test
+	public void testScheduleSourceBeforeTarget() throws Exception {
+
+		//                                            [pipelined]
+		//  we construct a simple graph    (source) ----------------> (target)
+
+		final int parallelism = 1;
+
+		final JobVertex sourceVertex = new JobVertex("source");
+		sourceVertex.setParallelism(parallelism);
+		sourceVertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobVertex targetVertex = new JobVertex("target");
+		targetVertex.setParallelism(parallelism);
+		targetVertex.setInvokableClass(NoOpInvokable.class);
+
+		targetVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph);
+
+		//
+		//  set up two TaskManager gateways and slots
+
+		final TaskManagerGateway gatewaySource = createTaskManager();
+		final TaskManagerGateway gatewayTarget = createTaskManager();
+
+		final SimpleSlot sourceSlot = createSlot(gatewaySource, jobId);
+		final SimpleSlot targetSlot = createSlot(gatewayTarget, jobId);
+
+		final FlinkCompletableFuture<SimpleSlot> sourceFuture = new FlinkCompletableFuture<>();
+		final FlinkCompletableFuture<SimpleSlot> targetFuture = new FlinkCompletableFuture<>();
+
+		ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism);
+		slotProvider.addSlot(sourceVertex.getID(), 0, sourceFuture);
+		slotProvider.addSlot(targetVertex.getID(), 0, targetFuture);
+
+		eg.setScheduleMode(ScheduleMode.EAGER);
+		eg.setQueuedSchedulingAllowed(true);
+		eg.scheduleForExecution(slotProvider);
+
+		// job should be running
+		assertEquals(JobStatus.RUNNING, eg.getState());
+
+		// we fulfill the target slot before the source slot
+		// that should not cause a deployment or deployment related failure
+		targetFuture.complete(targetSlot);
+
+		verify(gatewayTarget, new Timeout(50, times(0))).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+		assertEquals(JobStatus.RUNNING, eg.getState());
+
+		// now supply the source slot
+		sourceFuture.complete(sourceSlot);
+
+		// by now, all deployments should have happened
+		verify(gatewaySource, timeout(1000)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+		verify(gatewayTarget, timeout(1000)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+
+		assertEquals(JobStatus.RUNNING, eg.getState());
+	}
+
+	/**
+	 * This test verifies that before deploying a pipelined connected component, the
+	 * full set of slots is available, and that not some tasks are deployed, and later the
+	 * system realizes that not enough resources are available.
+	 */
+	@Test
+	public void testDeployPipelinedConnectedComponentsTogether() throws Exception {
+
+		//                                            [pipelined]
+		//  we construct a simple graph    (source) ----------------> (target)
+
+		final int parallelism = 8;
+
+		final JobVertex sourceVertex = new JobVertex("source");
+		sourceVertex.setParallelism(parallelism);
+		sourceVertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobVertex targetVertex = new JobVertex("target");
+		targetVertex.setParallelism(parallelism);
+		targetVertex.setInvokableClass(NoOpInvokable.class);
+
+		targetVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph);
+
+		//
+		//  Create the slots, futures, and the slot provider
+
+		final TaskManagerGateway[] sourceTaskManagers = new TaskManagerGateway[parallelism];
+		final TaskManagerGateway[] targetTaskManagers = new TaskManagerGateway[parallelism];
+
+		final SimpleSlot[] sourceSlots = new SimpleSlot[parallelism];
+		final SimpleSlot[] targetSlots = new SimpleSlot[parallelism];
+
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final FlinkCompletableFuture<SimpleSlot>[] sourceFutures = new FlinkCompletableFuture[parallelism];
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final FlinkCompletableFuture<SimpleSlot>[] targetFutures = new FlinkCompletableFuture[parallelism];
+
+		for (int i = 0; i < parallelism; i++) {
+			sourceTaskManagers[i] = createTaskManager();
+			targetTaskManagers[i] = createTaskManager();
+
+			sourceSlots[i] = createSlot(sourceTaskManagers[i], jobId);
+			targetSlots[i] = createSlot(targetTaskManagers[i], jobId);
+
+			sourceFutures[i] = new FlinkCompletableFuture<>();
+			targetFutures[i] = new FlinkCompletableFuture<>();
+		}
+
+		ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism);
+		slotProvider.addSlots(sourceVertex.getID(), sourceFutures);
+		slotProvider.addSlots(targetVertex.getID(), targetFutures);
+
+		//
+		//  we complete some of the futures
+
+		for (int i = 0; i < parallelism; i += 2) {
+			sourceFutures[i].complete(sourceSlots[i]);
+		}
+
+		//
+		//  kick off the scheduling
+
+		eg.setScheduleMode(ScheduleMode.EAGER);
+		eg.setQueuedSchedulingAllowed(true);
+		eg.scheduleForExecution(slotProvider);
+
+		verifyNothingDeployed(eg, sourceTaskManagers);
+
+		//  complete the remaining sources
+		for (int i = 1; i < parallelism; i += 2) {
+			sourceFutures[i].complete(sourceSlots[i]);
+		}
+		verifyNothingDeployed(eg, sourceTaskManagers);
+
+		//  complete the targets except for one
+		for (int i = 1; i < parallelism; i++) {
+			targetFutures[i].complete(targetSlots[i]);
+		}
+		verifyNothingDeployed(eg, targetTaskManagers);
+
+		//  complete the last target slot future
+		targetFutures[0].complete(targetSlots[0]);
+
+		//
+		//  verify that all deployments have happened
+
+		for (TaskManagerGateway gateway : sourceTaskManagers) {
+			verify(gateway, timeout(50)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+		}
+		for (TaskManagerGateway gateway : targetTaskManagers) {
+			verify(gateway, timeout(50)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+		}
+	}
+
+	/**
+	 * This test verifies that if one slot future fails, the deployment will be aborted.
+	 */
+	@Test
+	public void testOneSlotFailureAbortsDeploy() throws Exception {
+
+		//                                            [pipelined]
+		//  we construct a simple graph    (source) ----------------> (target)
+
+		final int parallelism = 6;
+
+		final JobVertex sourceVertex = new JobVertex("source");
+		sourceVertex.setParallelism(parallelism);
+		sourceVertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobVertex targetVertex = new JobVertex("target");
+		targetVertex.setParallelism(parallelism);
+		targetVertex.setInvokableClass(NoOpInvokable.class);
+
+		targetVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph);
+		TerminalJobStatusListener testListener = new TerminalJobStatusListener();
+		eg.registerJobStatusListener(testListener);
+
+		//
+		//  Create the slots, futures, and the slot provider
+
+		final TaskManagerGateway taskManager = mock(TaskManagerGateway.class);
+		final SlotOwner slotOwner = mock(SlotOwner.class);
+
+		final SimpleSlot[] sourceSlots = new SimpleSlot[parallelism];
+		final SimpleSlot[] targetSlots = new SimpleSlot[parallelism];
+
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final FlinkCompletableFuture<SimpleSlot>[] sourceFutures = new FlinkCompletableFuture[parallelism];
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final FlinkCompletableFuture<SimpleSlot>[] targetFutures = new FlinkCompletableFuture[parallelism];
+
+		for (int i = 0; i < parallelism; i++) {
+			sourceSlots[i] = createSlot(taskManager, jobId, slotOwner);
+			targetSlots[i] = createSlot(taskManager, jobId, slotOwner);
+
+			sourceFutures[i] = new FlinkCompletableFuture<>();
+			targetFutures[i] = new FlinkCompletableFuture<>();
+		}
+
+		ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism);
+		slotProvider.addSlots(sourceVertex.getID(), sourceFutures);
+		slotProvider.addSlots(targetVertex.getID(), targetFutures);
+
+		//
+		//  we complete some of the futures
+
+		for (int i = 0; i < parallelism; i += 2) {
+			sourceFutures[i].complete(sourceSlots[i]);
+			targetFutures[i + 1].complete(targetSlots[i + 1]);
+		}
+
+		//
+		//  kick off the scheduling
+
+		eg.setScheduleMode(ScheduleMode.EAGER);
+		eg.setQueuedSchedulingAllowed(true);
+		eg.scheduleForExecution(slotProvider);
+
+		// fail one slot
+		sourceFutures[1].completeExceptionally(new TestRuntimeException());
+
+		// wait until the job failed as a whole
+		testListener.waitForTerminalState(2000);
+
+		// wait until all slots are back
+		verify(slotOwner, new Timeout(2000, times(6))).returnAllocatedSlot(any(Slot.class));
+
+		// no deployment calls must have happened
+		verify(taskManager, times(0)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+
+		// all completed futures must have been returns
+		for (int i = 0; i < parallelism; i += 2) {
+			assertTrue(sourceSlots[i].isCanceled());
+			assertTrue(targetSlots[i + 1].isCanceled());
+		}
+	}
+
+	/**
+	 * This test verifies that the slot allocations times out after a certain time, and that
+	 * all slots are released in that case.
+	 */
+	@Test
+	public void testTimeoutForSlotAllocation() throws Exception {
+
+		//  we construct a simple graph:    (task)
+
+		final int parallelism = 3;
+
+		final JobVertex vertex = new JobVertex("task");
+		vertex.setParallelism(parallelism);
+		vertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", vertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph, Time.milliseconds(20));
+		final TerminalJobStatusListener statusListener = new TerminalJobStatusListener();
+		eg.registerJobStatusListener(statusListener);
+
+		final SlotOwner slotOwner = mock(SlotOwner.class);
+
+		final TaskManagerGateway taskManager = mock(TaskManagerGateway.class);
+		final SimpleSlot[] slots = new SimpleSlot[parallelism];
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final FlinkCompletableFuture<SimpleSlot>[] slotFutures = new FlinkCompletableFuture[parallelism];
+
+		for (int i = 0; i < parallelism; i++) {
+			slots[i] = createSlot(taskManager, jobId, slotOwner);
+			slotFutures[i] = new FlinkCompletableFuture<>();
+		}
+
+		ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism);
+		slotProvider.addSlots(vertex.getID(), slotFutures);
+
+		//  we complete one future
+		slotFutures[1].complete(slots[1]);
+
+		//  kick off the scheduling
+
+		eg.setScheduleMode(ScheduleMode.EAGER);
+		eg.setQueuedSchedulingAllowed(true);
+		eg.scheduleForExecution(slotProvider);
+
+		//  we complete another future
+		slotFutures[2].complete(slots[2]);
+
+		// since future[0] is still missing the while operation must time out
+		// we have no restarts allowed, so the job will go terminal
+		statusListener.waitForTerminalState(2000);
+
+		// wait until all slots are back
+		verify(slotOwner, new Timeout(2000, times(2))).returnAllocatedSlot(any(Slot.class));
+
+		//  verify that no deployments have happened
+		verify(taskManager, times(0)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+
+		for (Future<SimpleSlot> future : slotFutures) {
+			if (future.isDone()) {
+				assertTrue(future.get().isCanceled());
+			}
+		}
+	}
+
+	/**
+	 * Tests that the {@link ExecutionJobVertex#allocateResourcesForAll(SlotProvider, boolean)} method
+	 * releases partially acquired resources upon exception.
+	 */
+	@Test
+	public void testExecutionJobVertexAllocateResourcesReleasesOnException() throws Exception {
+		final int parallelism = 8;
+
+		final JobVertex vertex = new JobVertex("vertex");
+		vertex.setParallelism(parallelism);
+		vertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", vertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph);
+		final ExecutionJobVertex ejv = eg.getJobVertex(vertex.getID());
+
+		// set up some available slots and some slot owner that accepts released slots back
+		final List<SimpleSlot> returnedSlots = new ArrayList<>();
+		final SlotOwner recycler = new SlotOwner() {
+			@Override
+			public boolean returnAllocatedSlot(Slot slot) {
+				returnedSlots.add((SimpleSlot) slot);
+				return true;
+			}
+		};
+
+		final TaskManagerGateway taskManager = mock(TaskManagerGateway.class);
+		final List<SimpleSlot> availableSlots = new ArrayList<>(Arrays.asList(
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler)));
+
+
+		// slot provider that hand out parallelism / 3 slots, then throws an exception
+		final SlotProvider slots = mock(SlotProvider.class);
+		
+		when(slots.allocateSlot(any(ScheduledUnit.class), anyBoolean())).then(
+				new Answer<Future<SimpleSlot>>() {
+
+					@Override
+					public Future<SimpleSlot> answer(InvocationOnMock invocation) {
+						if (availableSlots.isEmpty()) {
+							throw new TestRuntimeException();
+						} else {
+							return FlinkCompletableFuture.completed(availableSlots.remove(0));
+						}
+					}
+				});
+
+		// acquire resources and check that all are back after the failure
+
+		final int numSlotsToExpectBack = availableSlots.size();
+
+		try {
+			ejv.allocateResourcesForAll(slots, false);
+			fail("should have failed with an exception");
+		}
+		catch (TestRuntimeException e) {
+			// expected
+		}
+
+		assertEquals(numSlotsToExpectBack, returnedSlots.size());
+	}
+
+	/**
+	 * Tests that the {@link ExecutionGraph#scheduleForExecution(SlotProvider)} method
+	 * releases partially acquired resources upon exception.
+	 */
+	@Test
+	public void testExecutionGraphScheduleReleasesResourcesOnException() throws Exception {
+
+		//                                            [pipelined]
+		//  we construct a simple graph    (source) ----------------> (target)
+
+		final int parallelism = 3;
+
+		final JobVertex sourceVertex = new JobVertex("source");
+		sourceVertex.setParallelism(parallelism);
+		sourceVertex.setInvokableClass(NoOpInvokable.class);
+
+		final JobVertex targetVertex = new JobVertex("target");
+		targetVertex.setParallelism(parallelism);
+		targetVertex.setInvokableClass(NoOpInvokable.class);
+
+		targetVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED);
+
+		final JobID jobId = new JobID();
+		final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex);
+
+		final ExecutionGraph eg = createExecutionGraph(jobGraph);
+
+		// set up some available slots and some slot owner that accepts released slots back
+		final List<SimpleSlot> returnedSlots = new ArrayList<>();
+		final SlotOwner recycler = new SlotOwner() {
+			@Override
+			public boolean returnAllocatedSlot(Slot slot) {
+				returnedSlots.add((SimpleSlot) slot);
+				return true;
+			}
+		};
+
+		final TaskManagerGateway taskManager = mock(TaskManagerGateway.class);
+		final List<SimpleSlot> availableSlots = new ArrayList<>(Arrays.asList(
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler),
+				createSlot(taskManager, jobId, recycler)));
+
+
+		// slot provider that hand out parallelism / 3 slots, then throws an exception
+		final SlotProvider slots = mock(SlotProvider.class);
+
+		when(slots.allocateSlot(any(ScheduledUnit.class), anyBoolean())).then(
+				new Answer<Future<SimpleSlot>>() {
+
+					@Override
+					public Future<SimpleSlot> answer(InvocationOnMock invocation) {
+						if (availableSlots.isEmpty()) {
+							throw new TestRuntimeException();
+						} else {
+							return FlinkCompletableFuture.completed(availableSlots.remove(0));
+						}
+					}
+				});
+
+		// acquire resources and check that all are back after the failure
+
+		final int numSlotsToExpectBack = availableSlots.size();
+
+		try {
+			eg.setScheduleMode(ScheduleMode.EAGER);
+			eg.scheduleForExecution(slots);
+			fail("should have failed with an exception");
+		}
+		catch (TestRuntimeException e) {
+			// expected
+		}
+
+		assertEquals(numSlotsToExpectBack, returnedSlots.size());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private ExecutionGraph createExecutionGraph(JobGraph jobGraph) throws Exception {
+		return createExecutionGraph(jobGraph, Time.minutes(10));
+	}
+
+	private ExecutionGraph createExecutionGraph(JobGraph jobGraph, Time timeout) throws Exception {
+		return ExecutionGraphBuilder.buildGraph(
+				null,
+				jobGraph,
+				new Configuration(),
+				executor,
+				executor,
+				getClass().getClassLoader(),
+				new StandaloneCheckpointRecoveryFactory(),
+				timeout,
+				new NoRestartStrategy(),
+				new UnregisteredMetricsGroup(),
+				1,
+				log);
+	}
+
+	private SimpleSlot createSlot(TaskManagerGateway taskManager, JobID jobId) {
+		return createSlot(taskManager, jobId, mock(SlotOwner.class));
+	}
+
+	private SimpleSlot createSlot(TaskManagerGateway taskManager, JobID jobId, SlotOwner slotOwner) {
+		TaskManagerLocation location = new TaskManagerLocation(
+				ResourceID.generate(), InetAddress.getLoopbackAddress(), 12345);
+
+		AllocatedSlot slot = new AllocatedSlot(
+				new AllocationID(), jobId, location, 0, ResourceProfile.UNKNOWN, taskManager);
+
+		return new SimpleSlot(slot, slotOwner, 0);
+	}
+
+	private static TaskManagerGateway createTaskManager() {
+		TaskManagerGateway tm = mock(TaskManagerGateway.class);
+		when(tm.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)))
+				.thenReturn(FlinkCompletableFuture.completed(Acknowledge.get()));
+
+		return tm;
+	}
+
+	private static void verifyNothingDeployed(ExecutionGraph eg, TaskManagerGateway[] taskManagers) {
+		// job should still be running
+		assertEquals(JobStatus.RUNNING, eg.getState());
+
+		// none of the TaskManager should have gotten a deployment call, yet
+		for (TaskManagerGateway gateway : taskManagers) {
+			verify(gateway, new Timeout(50, times(0))).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class));
+		}
+	}
+
+	private static class TestRuntimeException extends RuntimeException {
+		private static final long serialVersionUID = 1L;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtilsTest.java
new file mode 100644
index 0000000..2e6da98
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtilsTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.SlotOwner;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the utility methods in the class {@link ExecutionGraphUtils}.
+ */
+public class ExecutionGraphUtilsTest {
+
+	@Test
+	public void testReleaseSlots() {
+		final JobID jid = new JobID();
+		final SlotOwner owner = mock(SlotOwner.class);
+
+		final SimpleSlot slot1 = new SimpleSlot(createAllocatedSlot(jid, 0), owner, 0);
+		final SimpleSlot slot2 = new SimpleSlot(createAllocatedSlot(jid, 1), owner, 1);
+		final SimpleSlot slot3 = new SimpleSlot(createAllocatedSlot(jid, 2), owner, 2);
+
+		final FlinkCompletableFuture<SimpleSlot> incompleteFuture = new FlinkCompletableFuture<>();
+
+		final FlinkCompletableFuture<SimpleSlot> completeFuture = new FlinkCompletableFuture<>();
+		completeFuture.complete(slot2);
+
+		final FlinkCompletableFuture<SimpleSlot> disposedSlotFuture = new FlinkCompletableFuture<>();
+		slot3.releaseSlot();
+		disposedSlotFuture.complete(slot3);
+
+		// release all futures
+		ExecutionGraphUtils.releaseSlotFuture(incompleteFuture);
+		ExecutionGraphUtils.releaseSlotFuture(completeFuture);
+		ExecutionGraphUtils.releaseSlotFuture(disposedSlotFuture);
+
+		// only now complete the incomplete future
+		incompleteFuture.complete(slot1);
+
+		// verify that each slot was returned once to the owner
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot1));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot2));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot3));
+	}
+
+	@Test
+	public void testReleaseSlotsWithNulls() {
+		final JobID jid = new JobID();
+		final SlotOwner owner = mock(SlotOwner.class);
+
+		final Execution mockExecution = mock(Execution.class);
+
+		final SimpleSlot slot1 = new SimpleSlot(createAllocatedSlot(jid, 0), owner, 0);
+		final SimpleSlot slot2 = new SimpleSlot(createAllocatedSlot(jid, 1), owner, 1);
+		final SimpleSlot slot3 = new SimpleSlot(createAllocatedSlot(jid, 2), owner, 2);
+		final SimpleSlot slot4 = new SimpleSlot(createAllocatedSlot(jid, 3), owner, 3);
+		final SimpleSlot slot5 = new SimpleSlot(createAllocatedSlot(jid, 4), owner, 4);
+
+		ExecutionAndSlot[] slots1 = new ExecutionAndSlot[] {
+				null,
+				new ExecutionAndSlot(mockExecution, FlinkCompletableFuture.completed(slot1)),
+				null,
+				new ExecutionAndSlot(mockExecution, FlinkCompletableFuture.completed(slot2)),
+				null
+		};
+
+		ExecutionAndSlot[] slots2 = new ExecutionAndSlot[] {
+				new ExecutionAndSlot(mockExecution, FlinkCompletableFuture.completed(slot3)),
+				new ExecutionAndSlot(mockExecution, FlinkCompletableFuture.completed(slot4)),
+				new ExecutionAndSlot(mockExecution, FlinkCompletableFuture.completed(slot5))
+		};
+
+		List<ExecutionAndSlot[]> resources = Arrays.asList(null, slots1, new ExecutionAndSlot[0], null, slots2);
+
+		ExecutionGraphUtils.releaseAllSlotsSilently(resources);
+
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot1));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot2));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot3));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot4));
+		verify(owner, times(1)).returnAllocatedSlot(eq(slot5));
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static AllocatedSlot createAllocatedSlot(JobID jid, int num) {
+		TaskManagerLocation loc = new TaskManagerLocation(
+				ResourceID.generate(), InetAddress.getLoopbackAddress(), 10000 + num);
+	
+		return new AllocatedSlot(new AllocationID(), jid, loc, num,
+				ResourceProfile.UNKNOWN, mock(TaskManagerGateway.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index 7b6c6ea..82561b2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -457,7 +457,7 @@ public class ExecutionVertexCancelTest {
 			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 
 			// 1)
-			// scheduling after being created should be tolerated (no exception) because
+			// scheduling after being canceled should be tolerated (no exception) because
 			// it can occur as the result of races
 			{
 				Scheduler scheduler = mock(Scheduler.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
index 9132aee..1b029e8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
@@ -104,9 +104,6 @@ public class ExecutionVertexSchedulingTest {
 
 			future.complete(slot);
 
-			// wait a second for future's future action be executed
-			Thread.sleep(1000);
-
 			// will have failed
 			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
index 3a7e759..006f894 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
@@ -348,9 +348,9 @@ public class PointwisePatternTest {
 			
 			timesUsed[inEdges[0].getSource().getPartitionNumber()]++;
 		}
-		
-		for (int i = 0; i < timesUsed.length; i++) {
-			assertTrue(timesUsed[i] >= factor && timesUsed[i] <= factor + delta);
+
+		for (int used : timesUsed) {
+			assertTrue(used >= factor && used <= factor + delta);
 		}
 	}
 	
@@ -406,9 +406,9 @@ public class PointwisePatternTest {
 				timesUsed[ee.getSource().getPartitionNumber()]++;
 			}
 		}
-		
-		for (int i = 0; i < timesUsed.length; i++) {
-			assertEquals(1, timesUsed[i]);
+
+		for (int used : timesUsed) {
+			assertEquals(1, used);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java
new file mode 100644
index 0000000..3acb2eb
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.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.executiongraph;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.SlotProvider;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A slot provider where one can pre-set the slot futures for tasks based on
+ * vertex ID and subtask index.
+ */
+class ProgrammedSlotProvider implements SlotProvider {
+
+	private final Map<JobVertexID, Future<SimpleSlot>[]> slotFutures = new HashMap<>();
+
+	private final int parallelism;
+
+	public ProgrammedSlotProvider(int parallelism) {
+		checkArgument(parallelism > 0);
+		this.parallelism = parallelism;
+	}
+
+	public void addSlot(JobVertexID vertex, int subtaskIndex, Future<SimpleSlot> future) {
+		checkNotNull(vertex);
+		checkNotNull(future);
+		checkArgument(subtaskIndex >= 0 && subtaskIndex < parallelism);
+
+		Future<SimpleSlot>[] futures = slotFutures.get(vertex);
+		if (futures == null) {
+			@SuppressWarnings("unchecked")
+			Future<SimpleSlot>[] newArray = (Future<SimpleSlot>[]) new Future<?>[parallelism];
+			futures = newArray;
+			slotFutures.put(vertex, futures);
+		}
+
+		futures[subtaskIndex] = future;
+	}
+
+	public void addSlots(JobVertexID vertex, Future<SimpleSlot>[] futures) {
+		checkNotNull(vertex);
+		checkNotNull(futures);
+		checkArgument(futures.length == parallelism);
+
+		slotFutures.put(vertex, futures);
+	}
+
+	@Override
+	public Future<SimpleSlot> allocateSlot(ScheduledUnit task, boolean allowQueued) {
+		JobVertexID vertexId = task.getTaskToExecute().getVertex().getJobvertexId();
+		int subtask = task.getTaskToExecute().getParallelSubtaskIndex();
+
+		Future<SimpleSlot>[] forTask = slotFutures.get(vertexId);
+		if (forTask != null) {
+			Future<SimpleSlot> future = forTask[subtask];
+			if (future != null) {
+				return future;
+			}
+		}
+
+		throw new IllegalArgumentException("No registered slot future for task " + vertexId + " (" + subtask + ')');
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f113d794/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalJobStatusListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalJobStatusListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalJobStatusListener.java
new file mode 100644
index 0000000..c107d54
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalJobStatusListener.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.runtime.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A job status listener that waits lets one block until the job is in a terminal state.
+ */
+public class TerminalJobStatusListener  implements JobStatusListener {
+
+	private final OneShotLatch terminalStateLatch = new OneShotLatch();
+
+	public void waitForTerminalState(long timeoutMillis) throws InterruptedException, TimeoutException {
+		terminalStateLatch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+	}
+
+	@Override
+	public void jobStatusChanges(JobID jobId, JobStatus newJobStatus, long timestamp, Throwable error) {
+		if (newJobStatus.isGloballyTerminalState() || newJobStatus == JobStatus.SUSPENDED) {
+			terminalStateLatch.trigger();
+		}
+	}
+}


[17/19] flink git commit: [hotfix] [core] Add missing @Internal annotations to classes in flink-core.

Posted by se...@apache.org.
[hotfix] [core] Add missing @Internal annotations to classes in flink-core.


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

Branch: refs/heads/master
Commit: 082d40fdc415952737a109c913a86d009ed234c1
Parents: 0aa9918
Author: Fabian Hueske <fh...@apache.org>
Authored: Wed Feb 15 15:25:15 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:24 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/api/common/ArchivedExecutionConfig.java    | 3 +++
 .../api/common/typeutils/TypeSerializerSerializationProxy.java  | 4 +++-
 .../apache/flink/api/common/typeutils/base/ListSerializer.java  | 2 ++
 .../apache/flink/api/java/typeutils/runtime/AvroSerializer.java | 2 ++
 .../api/java/typeutils/runtime/CopyableValueComparator.java     | 2 ++
 .../api/java/typeutils/runtime/CopyableValueSerializer.java     | 2 ++
 .../flink/api/java/typeutils/runtime/DataInputDecoder.java      | 3 ++-
 .../flink/api/java/typeutils/runtime/DataInputViewStream.java   | 2 ++
 .../flink/api/java/typeutils/runtime/DataOutputEncoder.java     | 3 ++-
 .../flink/api/java/typeutils/runtime/DataOutputViewStream.java  | 2 ++
 .../flink/api/java/typeutils/runtime/EitherSerializer.java      | 2 ++
 .../flink/api/java/typeutils/runtime/FieldSerializer.java       | 3 +++
 .../flink/api/java/typeutils/runtime/GenericTypeComparator.java | 2 ++
 .../flink/api/java/typeutils/runtime/NoFetchingInput.java       | 2 ++
 .../flink/api/java/typeutils/runtime/NullAwareComparator.java   | 2 ++
 .../apache/flink/api/java/typeutils/runtime/NullMaskUtils.java  | 2 ++
 .../apache/flink/api/java/typeutils/runtime/PojoComparator.java | 3 ++-
 .../apache/flink/api/java/typeutils/runtime/PojoSerializer.java | 2 ++
 .../apache/flink/api/java/typeutils/runtime/RowComparator.java  | 2 ++
 .../apache/flink/api/java/typeutils/runtime/RowSerializer.java  | 2 ++
 .../api/java/typeutils/runtime/RuntimeComparatorFactory.java    | 2 ++
 .../java/typeutils/runtime/RuntimePairComparatorFactory.java    | 2 ++
 .../api/java/typeutils/runtime/RuntimeSerializerFactory.java    | 2 ++
 .../flink/api/java/typeutils/runtime/Tuple0Serializer.java      | 2 ++
 .../flink/api/java/typeutils/runtime/TupleComparator.java       | 3 ++-
 .../flink/api/java/typeutils/runtime/TupleComparatorBase.java   | 3 ++-
 .../flink/api/java/typeutils/runtime/TupleSerializer.java       | 3 ++-
 .../flink/api/java/typeutils/runtime/TupleSerializerBase.java   | 2 ++
 .../flink/api/java/typeutils/runtime/ValueComparator.java       | 2 ++
 .../flink/api/java/typeutils/runtime/ValueSerializer.java       | 2 ++
 .../apache/flink/core/fs/AbstractMultiFSDataInputStream.java    | 2 ++
 .../java/org/apache/flink/core/fs/ClosingFSDataInputStream.java | 4 +++-
 .../org/apache/flink/core/fs/ClosingFSDataOutputStream.java     | 4 +++-
 .../java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java | 4 +++-
 .../org/apache/flink/core/fs/FSDataOutputStreamWrapper.java     | 4 +++-
 .../org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java    | 4 +++-
 .../java/org/apache/flink/core/fs/WrappingProxyCloseable.java   | 2 ++
 .../apache/flink/core/memory/ByteArrayOutputStreamWithPos.java  | 2 ++
 .../java/org/apache/flink/util/AbstractCloseableRegistry.java   | 3 +++
 .../src/main/java/org/apache/flink/util/CollectionUtil.java     | 3 +++
 flink-core/src/main/java/org/apache/flink/util/FutureUtil.java  | 5 ++++-
 flink-core/src/main/java/org/apache/flink/util/Migration.java   | 3 +++
 .../src/main/java/org/apache/flink/util/WrappingProxy.java      | 3 +++
 .../src/main/java/org/apache/flink/util/WrappingProxyUtil.java  | 5 ++++-
 44 files changed, 104 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/common/ArchivedExecutionConfig.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ArchivedExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ArchivedExecutionConfig.java
index faf920d..f267e91 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ArchivedExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ArchivedExecutionConfig.java
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.api.common;
 
+import org.apache.flink.annotation.Internal;
+
 import java.io.Serializable;
 import java.util.Collections;
 import java.util.Map;
@@ -26,6 +28,7 @@ import java.util.Map;
  * It can be used to display job information on the web interface
  * without having to keep the classloader around after job completion.
  */
+@Internal
 public class ArchivedExecutionConfig implements Serializable {
 
 	private final String executionMode;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java
index 06ad8bf..cebd348 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.common.typeutils;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.io.VersionedIOReadableWritable;
 import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
 import org.apache.flink.core.memory.DataInputView;
@@ -30,6 +31,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.Arrays;
 
+@Internal
 public class TypeSerializerSerializationProxy<T> extends VersionedIOReadableWritable {
 
 	public static final int VERSION = 1;
@@ -228,4 +230,4 @@ public class TypeSerializerSerializationProxy<T> extends VersionedIOReadableWrit
 			return Arrays.hashCode(getActualBytes());
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
index ca3c143..5d5e8f8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.common.typeutils.base;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -37,6 +38,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * 
  * @param <T> The type of element in the list.
  */
+@Internal
 public class ListSerializer<T> extends TypeSerializer<List<T>> {
 
 	private static final long serialVersionUID = 1119562170939152304L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
index 4c2a7f9..8bcd157 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
@@ -25,6 +25,7 @@ import org.apache.avro.reflect.ReflectDatumReader;
 import org.apache.avro.reflect.ReflectDatumWriter;
 import org.apache.avro.util.Utf8;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
 import org.apache.flink.core.memory.DataInputView;
@@ -42,6 +43,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * @param <T> The type serialized.
  */
+@Internal
 public final class AvroSerializer<T> extends TypeSerializer<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
index 9b3b191..0093436 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -31,6 +32,7 @@ import org.apache.flink.util.InstantiationUtil;
 /**
  * Comparator for all Value types that extend Key
  */
+@Internal
 public class CopyableValueComparator<T extends CopyableValue<T> & Comparable<T>> extends TypeComparator<T> {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
index f30a767..1f3fcbc 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -28,6 +29,7 @@ import org.apache.flink.util.InstantiationUtil;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
+@Internal
 public class CopyableValueSerializer<T extends CopyableValue<T>> extends TypeSerializer<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java
index e48f9fa..c0454c6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java
@@ -24,8 +24,9 @@ import java.nio.ByteBuffer;
 
 import org.apache.avro.io.Decoder;
 import org.apache.avro.util.Utf8;
+import org.apache.flink.annotation.Internal;
 
-
+@Internal
 public class DataInputDecoder extends Decoder implements java.io.Serializable {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java
index be17d64..aab4689 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.memory.DataInputView;
 
 import java.io.EOFException;
@@ -27,6 +28,7 @@ import java.io.InputStream;
 /**
  * An input stream that draws its data from a {@link DataInputView}.
  */
+@Internal
 public class DataInputViewStream extends InputStream {
 	
 	protected DataInputView inputView;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java
index 5c89962..c41b648 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java
@@ -24,8 +24,9 @@ import java.nio.ByteBuffer;
 
 import org.apache.avro.io.Encoder;
 import org.apache.avro.util.Utf8;
+import org.apache.flink.annotation.Internal;
 
-
+@Internal
 public final class DataOutputEncoder extends Encoder implements java.io.Serializable {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java
index 66f2af6..9c06043 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java
@@ -18,10 +18,12 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.memory.DataOutputView;
 import java.io.IOException;
 import java.io.OutputStream;
 
+@Internal
 public class DataOutputViewStream extends OutputStream {
 	protected DataOutputView outputView;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
index e5d9070..4066e9a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -34,6 +35,7 @@ import static org.apache.flink.types.Either.Right;
  * @param <L> the Left value type
  * @param <R> the Right value type
  */
+@Internal
 public class EitherSerializer<L, R> extends TypeSerializer<Either<L, R>> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/FieldSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/FieldSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/FieldSerializer.java
index 057eee9..5519889 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/FieldSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/FieldSerializer.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
+
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
@@ -28,6 +30,7 @@ import java.lang.reflect.Field;
  * readObject/writeObject need to be implemented in classes where there is a field of type java.lang.reflect.Field.
  * The two static methods in this class are to be called from these readObject/writeObject methods.
  */
+@Internal
 public class FieldSerializer {
 
 	public static void serializeField(Field field, ObjectOutputStream out) throws IOException {

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
index 28fea6a..e708079 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
@@ -32,6 +33,7 @@ import org.apache.flink.util.InstantiationUtil;
 /**
  * TypeComparator for all types that extend Comparable.
  */
+@Internal
 public class GenericTypeComparator<T extends Comparable<T>> extends TypeComparator<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java
index 0f4fe94..b247783 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java
@@ -20,11 +20,13 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import com.esotericsoftware.kryo.KryoException;
 import com.esotericsoftware.kryo.io.Input;
+import org.apache.flink.annotation.Internal;
 
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 
+@Internal
 public class NoFetchingInput extends Input {
 	public NoFetchingInput(InputStream inputStream){
 		super(inputStream, 8);

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java
index 3587811..47297c7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeComparator;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.core.memory.DataInputView;
@@ -33,6 +34,7 @@ import java.util.List;
  * NOTE: This class assumes to be used within a composite type comparator (such
  * as {@link RowComparator}) that handles serialized comparison.
  */
+@Internal
 public class NullAwareComparator<T> extends TypeComparator<T> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java
index 010af7f..cfe562f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java
@@ -17,12 +17,14 @@
  */
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.Row;
 
 import java.io.IOException;
 
+@Internal
 public class NullMaskUtils {
 
 	public static void writeNullMask(int len, Row value, DataOutputView target) throws IOException {

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
index fc4a305..945abc8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
@@ -24,6 +24,7 @@ import java.io.ObjectOutputStream;
 import java.lang.reflect.Field;
 import java.util.List;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeComparator;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -33,7 +34,7 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.types.NullKeyFieldException;
 import org.apache.flink.util.InstantiationUtil;
 
-
+@Internal
 public final class PojoComparator<T> extends CompositeTypeComparator<T> implements java.io.Serializable {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
index 57928b8..1a9c8f9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
@@ -32,6 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -41,6 +42,7 @@ import org.apache.flink.core.memory.DataOutputView;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
+@Internal
 public final class PojoSerializer<T> extends TypeSerializer<T> {
 
 	// Flags for the header

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java
index d6c5195..135623b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeComparator;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -37,6 +38,7 @@ import static org.apache.flink.util.Preconditions.checkArgument;
 /**
  * Comparator for {@link Row}
  */
+@Internal
 public class RowComparator extends CompositeTypeComparator<Row> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
index 5457c05..827c2fa 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -33,6 +34,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 /**
  * Serializer for {@link Row}.
  */
+@Internal
 public class RowSerializer extends TypeSerializer<Row> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java
index 4b734a7..13d5575 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java
@@ -19,11 +19,13 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.InstantiationUtil;
 
+@Internal
 public final class RuntimeComparatorFactory<T> implements TypeComparatorFactory<T>, java.io.Serializable {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
index 31e28f7..13b3f2c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
@@ -18,11 +18,13 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 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.TypePairComparatorFactory;
 
+@Internal
 public final class RuntimePairComparatorFactory<T1, T2>
 		implements TypePairComparatorFactory<T1, T2>, java.io.Serializable {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java
index 96aff73..f03a7ab 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java
@@ -19,11 +19,13 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.InstantiationUtil;
 
+@Internal
 public final class RuntimeSerializerFactory<T> implements TypeSerializerFactory<T>, java.io.Serializable {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
index a06ff1a..b96126d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
@@ -14,11 +14,13 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple0;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
+@Internal
 public class Tuple0Serializer extends TupleSerializer<Tuple0> {
 	
 	private static final long serialVersionUID = 1278813169022975971L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
index 875ecc2..e5e32be 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple;
@@ -26,7 +27,7 @@ import org.apache.flink.types.KeyFieldOutOfBoundsException;
 import org.apache.flink.types.NullFieldException;
 import org.apache.flink.types.NullKeyFieldException;
 
-
+@Internal
 public final class TupleComparator<T extends Tuple> extends TupleComparatorBase<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
index 28169e5..f7a327a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeComparator;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -28,7 +29,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.KeyFieldOutOfBoundsException;
 import org.apache.flink.types.NullKeyFieldException;
 
-
+@Internal
 public abstract class TupleComparatorBase<T> extends CompositeTypeComparator<T> implements java.io.Serializable {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
index 0897063..df6694c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
@@ -20,13 +20,14 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.NullFieldException;
 
-
+@Internal
 public class TupleSerializer<T extends Tuple> extends TupleSerializerBase<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
index 5a93cc5..afc4aa2 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -28,6 +29,7 @@ import java.util.Objects;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
+@Internal
 public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
index 4b9629a..fb81bcd 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -34,6 +35,7 @@ import org.objenesis.strategy.StdInstantiatorStrategy;
 /**
  * Comparator for all Value types that extend Key
  */
+@Internal
 public class ValueComparator<T extends Value & Comparable<T>> extends TypeComparator<T> {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
index 73dc0fc..56e204c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -37,6 +38,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * @param <T> The type serialized.
  */
+@Internal
 public class ValueSerializer<T extends Value> extends TypeSerializer<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java
index 88c0092..a161ceb 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.Preconditions;
 
@@ -28,6 +29,7 @@ import java.io.IOException;
  * Abstract base class for wrappers over multiple {@link FSDataInputStream}, which gives a contiguous view on all inner
  * streams and makes them look like a single stream, in which we can read, seek, etc.
  */
+@Internal
 public abstract class AbstractMultiFSDataInputStream extends FSDataInputStream {
 
 	/** Inner stream for the currently accessed segment of the virtual global stream */

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java
index 23ac4f2..7c97271 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -28,6 +29,7 @@ import java.io.IOException;
  * <p>
  * See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
  */
+@Internal
 public class ClosingFSDataInputStream
 		extends FSDataInputStreamWrapper
 		implements WrappingProxyCloseable<FSDataInputStream> {
@@ -94,4 +96,4 @@ public class ClosingFSDataInputStream
 		registry.registerClosable(inputStream);
 		return inputStream;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java
index 120ca67..c517a83 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -28,6 +29,7 @@ import java.io.IOException;
  * <p>
  * See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
  */
+@Internal
 public class ClosingFSDataOutputStream
 		extends FSDataOutputStreamWrapper
 		implements WrappingProxyCloseable<FSDataOutputStream> {
@@ -99,4 +101,4 @@ public class ClosingFSDataOutputStream
 		registry.registerClosable(inputStream);
 		return inputStream;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java
index 507b756..d2eb9f2 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.WrappingProxy;
 
@@ -26,6 +27,7 @@ import java.io.IOException;
 /**
  * Simple forwarding wrapper around {@link FSDataInputStream}
  */
+@Internal
 public class FSDataInputStreamWrapper extends FSDataInputStream implements WrappingProxy<FSDataInputStream> {
 
 	protected final FSDataInputStream inputStream;
@@ -93,4 +95,4 @@ public class FSDataInputStreamWrapper extends FSDataInputStream implements Wrapp
 	public FSDataInputStream getWrappedDelegate() {
 		return inputStream;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java
index 36ebe10..f015012 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.WrappingProxy;
 
@@ -26,6 +27,7 @@ import java.io.IOException;
 /**
  * Simple forwarding wrapper around {@link FSDataInputStream}
  */
+@Internal
 public class FSDataOutputStreamWrapper extends FSDataOutputStream implements WrappingProxy<FSDataOutputStream> {
 
 	protected final FSDataOutputStream outputStream;
@@ -73,4 +75,4 @@ public class FSDataOutputStreamWrapper extends FSDataOutputStream implements Wra
 	public FSDataOutputStream getWrappedDelegate() {
 		return outputStream;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
index bf30b4f..63e6253 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.WrappingProxy;
 
@@ -32,6 +33,7 @@ import java.net.URI;
  * Streams obtained by this are therefore managed by the {@link SafetyNetCloseableRegistry} to prevent resource leaks
  * from unclosed streams.
  */
+@Internal
 public class SafetyNetWrapperFileSystem extends FileSystem implements WrappingProxy<FileSystem> {
 
 	private final SafetyNetCloseableRegistry registry;
@@ -147,4 +149,4 @@ public class SafetyNetWrapperFileSystem extends FileSystem implements WrappingPr
 	public FileSystem getWrappedDelegate() {
 		return unsafeFileSystem;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java b/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java
index b74fc78..9f100ef 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.WrappingProxy;
 
 import java.io.Closeable;
@@ -25,6 +26,7 @@ import java.io.Closeable;
 /**
  * {@link WrappingProxy} for {@link Closeable} that is also closeable.
  */
+@Internal
 public interface WrappingProxyCloseable<T extends Closeable> extends Closeable, WrappingProxy<T> {
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java
index ebaf1b9..ddfd30a 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.memory;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -27,6 +28,7 @@ import java.util.Arrays;
 /**
  * Un-synchronized stream similar to Java's ByteArrayOutputStream that also exposes the current position.
  */
+@Internal
 public class ByteArrayOutputStreamWithPos extends OutputStream {
 
 	protected byte[] buffer;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
index 85af982..e165d97 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Map;
@@ -33,6 +35,7 @@ import java.util.Map;
  * @param <C> Type of the closeable this registers
  * @param <T> Type for potential meta data associated with the registering closeables
  */
+@Internal
 public abstract class AbstractCloseableRegistry<C extends Closeable, T> implements Closeable {
 
 	protected final Map<Closeable, T> closeableToRef;

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
index cd5c2e5..bd6847a 100644
--- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
@@ -18,9 +18,12 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
 import java.util.Collection;
 import java.util.Map;
 
+@Internal
 public final class CollectionUtil {
 
 	private CollectionUtil() {

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java b/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
index 62d836b..5ecf344 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
@@ -18,9 +18,12 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.RunnableFuture;
 
+@Internal
 public class FutureUtil {
 
 	private FutureUtil() {
@@ -39,4 +42,4 @@ public class FutureUtil {
 
 		return future.get();
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/Migration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Migration.java b/flink-core/src/main/java/org/apache/flink/util/Migration.java
index 4bd9e39..a82488d 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Migration.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Migration.java
@@ -18,8 +18,11 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
 /**
  * Tagging interface for migration related classes.
  */
+@Internal
 public interface Migration {
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java b/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
index 82fcf04..213feee 100644
--- a/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
+++ b/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
+@Internal
 public interface WrappingProxy<T> {
 
 	T getWrappedDelegate();

http://git-wip-us.apache.org/repos/asf/flink/blob/082d40fd/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java b/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
index 0f62abd..6a79913 100644
--- a/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
+@Internal
 public final class WrappingProxyUtil {
 
 	private WrappingProxyUtil() {
@@ -30,4 +33,4 @@ public final class WrappingProxyUtil {
 		}
 		return object;
 	}
-}
\ No newline at end of file
+}


[13/19] flink git commit: [FLINK-5277] [tests] Add unit tests for ResultPartition#add() in case of failures

Posted by se...@apache.org.
[FLINK-5277] [tests] Add unit tests for ResultPartition#add() in case of failures

This verifies that the given network buffer is recycled as expected and that
no notifiers are called upon failures to add a buffer.

This closes #3309


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

Branch: refs/heads/master
Commit: 1ceb7d82eccf4dc77482bddb61a664fd7f226b2b
Parents: 5e32eb5
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Tue Feb 14 17:42:28 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../network/partition/ResultPartitionTest.java  | 75 ++++++++++++++++++++
 1 file changed, 75 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb7d82/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
index f6562a1..0cd3591 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -20,13 +20,16 @@ package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.util.TestBufferFactory;
 import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.junit.Assert;
 import org.junit.Test;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -70,6 +73,78 @@ public class ResultPartitionTest {
 		}
 	}
 
+	@Test
+	public void testAddOnFinishedPipelinedPartition() throws Exception {
+		testAddOnFinishedPartition(ResultPartitionType.PIPELINED);
+	}
+
+	@Test
+	public void testAddOnFinishedBlockingPartition() throws Exception {
+		testAddOnFinishedPartition(ResultPartitionType.BLOCKING);
+	}
+
+	/**
+	 * Tests {@link ResultPartition#add} on a partition which has already finished.
+	 *
+	 * @param pipelined the result partition type to set up
+	 */
+	protected void testAddOnFinishedPartition(final ResultPartitionType pipelined)
+		throws Exception {
+		Buffer buffer = TestBufferFactory.createBuffer();
+		ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+		try {
+			ResultPartition partition = createPartition(notifier, pipelined, true);
+			partition.finish();
+			reset(notifier);
+			// partition.add() should fail
+			partition.add(buffer, 0);
+			Assert.fail("exception expected");
+		} catch (IllegalStateException e) {
+			// expected => ignored
+		} finally {
+			if (!buffer.isRecycled()) {
+				Assert.fail("buffer not recycled");
+				buffer.recycle();
+			}
+			// should not have notified either
+			verify(notifier, never()).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+		}
+	}
+
+	@Test
+	public void testAddOnReleasedPipelinedPartition() throws Exception {
+		testAddOnReleasedPartition(ResultPartitionType.PIPELINED);
+	}
+
+	@Test
+	public void testAddOnReleasedBlockingPartition() throws Exception {
+		testAddOnReleasedPartition(ResultPartitionType.BLOCKING);
+	}
+
+	/**
+	 * Tests {@link ResultPartition#add} on a partition which has already been released.
+	 *
+	 * @param pipelined the result partition type to set up
+	 */
+	protected void testAddOnReleasedPartition(final ResultPartitionType pipelined)
+		throws Exception {
+		Buffer buffer = TestBufferFactory.createBuffer();
+		ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+		try {
+			ResultPartition partition = createPartition(notifier, pipelined, true);
+			partition.release();
+			// partition.add() silently drops the buffer but recycles it
+			partition.add(buffer, 0);
+		} finally {
+			if (!buffer.isRecycled()) {
+				Assert.fail("buffer not recycled");
+				buffer.recycle();
+			}
+			// should not have notified either
+			verify(notifier, never()).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+		}
+	}
+
 	// ------------------------------------------------------------------------
 
 	private static ResultPartition createPartition(


[19/19] flink git commit: [FLINK-5812] [core] Cleanups in FileSystem (round 2)

Posted by se...@apache.org.
[FLINK-5812] [core] Cleanups in FileSystem (round 2)

Move the FileSystem safety net to a separate class.


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

Branch: refs/heads/master
Commit: 5902ea0e88c70f330c23b9ace94033ae34c84445
Parents: a1bfae9
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Feb 15 17:58:37 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:24 2017 +0100

----------------------------------------------------------------------
 flink-core/pom.xml                              |   2 +-
 .../org/apache/flink/core/fs/FileSystem.java    |  52 +-------
 .../flink/core/fs/FileSystemSafetyNet.java      | 124 +++++++++++++++++++
 .../flink/util/AbstractCloseableRegistry.java   |   4 -
 .../core/fs/SafetyNetCloseableRegistryTest.java |   8 +-
 .../apache/flink/runtime/taskmanager/Task.java  |  11 +-
 6 files changed, 140 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index e9738a2..0a0d06e 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -154,7 +154,7 @@ under the License.
 					<parameter>
 						<excludes combine.children="append">
 							<exclude>org.apache.flink.api.common.ExecutionConfig#CONFIG_KEY</exclude>
-							<exclude>org.apache.flink.core.fs.FileSystem$FSKey</exclude>
+							<exclude>org.apache.flink.core.fs.FileSystem\$FSKey</exclude>
 							<exclude>org.apache.flink.api.java.typeutils.WritableTypeInfo</exclude>
 							<!-- Breaking changes between 1.1 and 1.2. 
 							We ignore these changes because these are low-level, internal runtime configuration parameters -->

http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/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 4149d5e..fab0f4d 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
@@ -17,7 +17,7 @@
  */
 
 
-/**
+/*
  * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache
  * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
  * additional information regarding copyright ownership.
@@ -30,12 +30,7 @@ import org.apache.flink.annotation.Public;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.local.LocalFileSystem;
-import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.OperatingSystem;
-import org.apache.flink.util.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 import java.io.File;
@@ -174,6 +169,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * application task finishes (or is canceled or failed). That way, the task's threads do not
  * leak connections.
  * 
+ * <p>Internal runtime code can explicitly obtain a FileSystem that does not use the safety
+ * net via {@link FileSystem#getUnguardedFileSystem(URI)}.
+ * 
  * @see FSDataInputStream
  * @see FSDataOutputStream
  */
@@ -198,57 +196,18 @@ public abstract class FileSystem {
 
 	// ------------------------------------------------------------------------
 
-	private static final ThreadLocal<SafetyNetCloseableRegistry> REGISTRIES = new ThreadLocal<>();
-
 	private static final String HADOOP_WRAPPER_FILESYSTEM_CLASS = "org.apache.flink.runtime.fs.hdfs.HadoopFileSystem";
 
 	private static final String MAPR_FILESYSTEM_CLASS = "org.apache.flink.runtime.fs.maprfs.MapRFileSystem";
 
 	private static final String HADOOP_WRAPPER_SCHEME = "hdwrapper";
 
-	private static final Logger LOG = LoggerFactory.getLogger(FileSystem.class);
-
 	/** This lock guards the methods {@link #initOutPathLocalFS(Path, WriteMode, boolean)} and
 	 * {@link #initOutPathDistFS(Path, WriteMode, boolean)} which are otherwise susceptible to races */
 	private static final ReentrantLock OUTPUT_DIRECTORY_INIT_LOCK = new ReentrantLock(true);
 
 	// ------------------------------------------------------------------------
 
-	/**
-	 * Create a SafetyNetCloseableRegistry for a Task. This method should be called at the beginning of the task's
-	 * main thread.
-	 */
-	@Internal
-	public static void createAndSetFileSystemCloseableRegistryForThread() {
-		SafetyNetCloseableRegistry oldRegistry = REGISTRIES.get();
-		Preconditions.checkState(null == oldRegistry,
-				"Found old CloseableRegistry " + oldRegistry +
-						". This indicates a leak of the InheritableThreadLocal through a ThreadPool!");
-
-		SafetyNetCloseableRegistry newRegistry = new SafetyNetCloseableRegistry();
-		REGISTRIES.set(newRegistry);
-		LOG.info("Created new CloseableRegistry " + newRegistry + " for {}", Thread.currentThread().getName());
-	}
-
-	/**
-	 * Create a SafetyNetCloseableRegistry for a Task. This method should be called at the end of the task's
-	 * main thread or when the task should be canceled.
-	 */
-	@Internal
-	public static void closeAndDisposeFileSystemCloseableRegistryForThread() {
-		SafetyNetCloseableRegistry registry = REGISTRIES.get();
-		if (null != registry) {
-			LOG.info("Ensuring all FileSystem streams are closed for {}", Thread.currentThread().getName());
-			REGISTRIES.remove();
-			IOUtils.closeQuietly(registry);
-		}
-	}
-
-	private static FileSystem wrapWithSafetyNetWhenActivated(FileSystem fs) {
-		SafetyNetCloseableRegistry reg = REGISTRIES.get();
-		return reg != null ? new SafetyNetWrapperFileSystem(fs, reg) : fs;
-	}
-
 	/** Object used to protect calls to specific methods.*/
 	private static final Object SYNCHRONIZATION_OBJECT = new Object();
 
@@ -427,7 +386,7 @@ public abstract class FileSystem {
 	 *         thrown if a reference to the file system instance could not be obtained
 	 */
 	public static FileSystem get(URI uri) throws IOException {
-		return wrapWithSafetyNetWhenActivated(getUnguardedFileSystem(uri));
+		return FileSystemSafetyNet.wrapWithSafetyNetWhenActivated(getUnguardedFileSystem(uri));
 	}
 
 	/**
@@ -971,7 +930,6 @@ public abstract class FileSystem {
 
 	/**
 	 * An identifier of a file system, via its scheme and its authority.
-	 * This class needs to stay public, because it is detected as part of the public API.
 	 */
 	private static final class FSKey {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java
new file mode 100644
index 0000000..b18cb13
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java
@@ -0,0 +1,124 @@
+/*
+ * 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.core.fs;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.IOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The FileSystemSafetyNet can be used to guard a thread against {@link FileSystem} stream resource leaks.
+ * When activated for a thread, it tracks all streams that are opened by FileSystems that the thread
+ * obtains. The safety net has a global cleanup hook that will close all streams that were
+ * not properly closed.
+ * 
+ * <p>The main thread of each Flink task, as well as the checkpointing thread are automatically guarded
+ * by this safety net.
+ * 
+ * <p><b>Important:</b> This safety net works only for streams created by Flink's FileSystem abstraction,
+ * i.e., for {@code FileSystem} instances obtained via {@link FileSystem#get(URI)} or through
+ * {@link Path#getFileSystem()}.
+ * 
+ * <p><b>Important:</b> When a guarded thread obtains a {@code FileSystem} or a stream and passes them
+ * to another thread, the safety net will close those resources once the former thread finishes.
+ * 
+ * <p>The safety net can be used as follows:
+ * <pre>{@code
+ * 
+ * class GuardedThread extends Thread {
+ * 
+ *     public void run() {
+ *         FileSystemSafetyNet.initializeSafetyNetForThread();
+ *         try {
+ *             // do some heavy stuff where you are unsure whether it closes all streams
+ *             // like some untrusted user code or library code
+ *         }
+ *         finally {
+ *             FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+ *         }
+ *     }
+ * }
+ * }</pre>
+ */
+@Internal
+public class FileSystemSafetyNet {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FileSystemSafetyNet.class);
+
+	/** The map from thread to the safety net registry for that thread */
+	private static final ThreadLocal<SafetyNetCloseableRegistry> REGISTRIES = new ThreadLocal<>();
+
+	// ------------------------------------------------------------------------
+	//  Activating / Deactivating
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Activates the safety net for a thread. {@link FileSystem} instances obtained by the thread
+	 * that called this method will be guarded, meaning that their created streams are tracked and can
+	 * be closed via the safety net closing hook.
+	 * 
+	 * <p>This method should be called at the beginning of a thread that should be guarded.
+	 * 
+	 * @throws IllegalStateException Thrown, if a safety net was already registered for the thread.
+	 */
+	@Internal
+	public static void initializeSafetyNetForThread() {
+		SafetyNetCloseableRegistry oldRegistry = REGISTRIES.get();
+
+		checkState(null == oldRegistry, "Found an existing FileSystem safety net for this thread: %s " +
+				"This may indicate an accidental repeated initialization, or a leak of the" +
+				"(Inheritable)ThreadLocal through a ThreadPool.", oldRegistry);
+
+		SafetyNetCloseableRegistry newRegistry = new SafetyNetCloseableRegistry();
+		REGISTRIES.set(newRegistry);
+		LOG.info("Created new CloseableRegistry {} for {}", newRegistry, Thread.currentThread().getName());
+	}
+
+	/**
+	 * Closes the safety net for a thread. This closes all remaining unclosed streams that were opened
+	 * by safety-net-guarded file systems. After this method was called, no streams can be opened any more
+	 * from any FileSystem instance that was obtained while the thread was guarded by the safety net.
+	 * 
+	 * <p>This method should be called at the very end of a guarded thread.
+	 */
+	@Internal
+	public static void closeSafetyNetAndGuardedResourcesForThread() {
+		SafetyNetCloseableRegistry registry = REGISTRIES.get();
+		if (null != registry) {
+			LOG.info("Ensuring all FileSystem streams are closed for {}", Thread.currentThread().getName());
+			REGISTRIES.remove();
+			IOUtils.closeQuietly(registry);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	static FileSystem wrapWithSafetyNetWhenActivated(FileSystem fs) {
+		SafetyNetCloseableRegistry reg = REGISTRIES.get();
+		return reg != null ? new SafetyNetWrapperFileSystem(fs, reg) : fs;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
index 2b7a8c8..766ede9 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
@@ -108,10 +108,6 @@ public abstract class AbstractCloseableRegistry<C extends Closeable, T> implemen
 		return closeableToRef;
 	}
 
-	// ------------------------------------------------------------------------
-	//  
-	// ------------------------------------------------------------------------
-
 	protected abstract void doUnRegister(C closeable, Map<Closeable, T> closeableMap);
 
 	protected abstract void doRegister(C closeable, Map<Closeable, T> closeableMap) throws IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java
index 6870780..7973c69 100644
--- a/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java
+++ b/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java
@@ -77,7 +77,7 @@ public class SafetyNetCloseableRegistryTest {
 					FileSystem fs1 = FileSystem.getLocalFileSystem();
 					// ensure no safety net in place
 					Assert.assertFalse(fs1 instanceof SafetyNetWrapperFileSystem);
-					FileSystem.createAndSetFileSystemCloseableRegistryForThread();
+					FileSystemSafetyNet.initializeSafetyNetForThread();
 					fs1 = FileSystem.getLocalFileSystem();
 					// ensure safety net is in place now
 					Assert.assertTrue(fs1 instanceof SafetyNetWrapperFileSystem);
@@ -91,11 +91,11 @@ public class SafetyNetCloseableRegistryTest {
 								FileSystem fs2 = FileSystem.getLocalFileSystem();
 								// ensure the safety net does not leak here
 								Assert.assertFalse(fs2 instanceof SafetyNetWrapperFileSystem);
-								FileSystem.createAndSetFileSystemCloseableRegistryForThread();
+								FileSystemSafetyNet.initializeSafetyNetForThread();
 								fs2 = FileSystem.getLocalFileSystem();
 								// ensure we can bring another safety net in place
 								Assert.assertTrue(fs2 instanceof SafetyNetWrapperFileSystem);
-								FileSystem.closeAndDisposeFileSystemCloseableRegistryForThread();
+								FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
 								fs2 = FileSystem.getLocalFileSystem();
 								// and that we can remove it again
 								Assert.assertFalse(fs2 instanceof SafetyNetWrapperFileSystem);
@@ -107,7 +107,7 @@ public class SafetyNetCloseableRegistryTest {
 
 						//ensure stream is still open and was never closed by any interferences
 						stream.write(42);
-						FileSystem.closeAndDisposeFileSystemCloseableRegistryForThread();
+						FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
 
 						// ensure leaking stream was closed
 						try {

http://git-wip-us.apache.org/repos/asf/flink/blob/5902ea0e/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 64a83c9..acb423b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemSafetyNet;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.blob.BlobKey;
@@ -552,7 +552,7 @@ public class Task implements Runnable, TaskActions {
 			// ----------------------------
 
 			// activate safety net for task thread
-			FileSystem.createAndSetFileSystemCloseableRegistryForThread();
+			FileSystemSafetyNet.initializeSafetyNetForThread();
 
 			// first of all, get a user-code classloader
 			// this may involve downloading the job's JAR files and/or classes
@@ -789,8 +789,9 @@ public class Task implements Runnable, TaskActions {
 
 				// remove all files in the distributed cache
 				removeCachedFiles(distributedCacheEntries, fileCache);
+
 				// close and de-activate safety net for task thread
-				FileSystem.closeAndDisposeFileSystemCloseableRegistryForThread();
+				FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
 
 				notifyFinalState();
 			}
@@ -1131,7 +1132,7 @@ public class Task implements Runnable, TaskActions {
 					@Override
 					public void run() {
 						// activate safety net for checkpointing thread
-						FileSystem.createAndSetFileSystemCloseableRegistryForThread();
+						FileSystemSafetyNet.initializeSafetyNetForThread();
 						try {
 							boolean success = statefulTask.triggerCheckpoint(checkpointMetaData);
 							if (!success) {
@@ -1152,7 +1153,7 @@ public class Task implements Runnable, TaskActions {
 							}
 						} finally {
 							// close and de-activate safety net for checkpointing thread
-							FileSystem.closeAndDisposeFileSystemCloseableRegistryForThread();
+							FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
 						}
 					}
 				};


[11/19] flink git commit: [FLINK-5669] [contrib] Change DataStreamUtils to use the loopback address (127.0.0.1) with local environments.

Posted by se...@apache.org.
[FLINK-5669] [contrib] Change DataStreamUtils to use the loopback address (127.0.0.1) with local environments.

Using loopback rather than the "local address" allows tests to run in
situations where the local machine's hostname may not be resolvable in DNS
(because DNS is unreacable or the hostname is not found) or the hostname does
resolve, but not to an IP address that is reachable.

This closes #3223


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

Branch: refs/heads/master
Commit: 3104619250fa0e0e87b4bb3e05b1cce9d39e6983
Parents: 4ce2557
Author: Rick Cox <ri...@amazon.com>
Authored: Thu Jan 26 14:55:23 2017 -0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Feb 20 01:01:23 2017 +0100

----------------------------------------------------------------------
 .../java/org/apache/flink/contrib/streaming/DataStreamUtils.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/31046192/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java
index d4ef9ee..2987597 100644
--- a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java
+++ b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java
@@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.net.ConnectionUtils;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
 import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
@@ -57,6 +58,8 @@ public final class DataStreamUtils {
 				throw new IOException("Could not determine an suitable network address to " +
 						"receive back data from the streaming program.", e);
 			}
+		} else if (env instanceof LocalStreamEnvironment) {
+			clientAddress = InetAddress.getLoopbackAddress();
 		} else {
 			try {
 				clientAddress = InetAddress.getLocalHost();