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/28 18:36:44 UTC

[01/11] flink git commit: [FLINK-5501] [runtime] Extend RunningJobRegistry to job status created/running/done

Repository: flink
Updated Branches:
  refs/heads/master daf0ccda4 -> 3086af534


[FLINK-5501] [runtime] Extend RunningJobRegistry to job status created/running/done

This closes #3385


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

Branch: refs/heads/master
Commit: e7011d78a3019880a4e00ab5f697c3cfd20161bb
Parents: 40743aa
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Feb 22 14:15:43 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:09 2017 +0100

----------------------------------------------------------------------
 .../FsNegativeRunningJobsRegistry.java          | 86 ++++++++++++++++----
 .../highavailability/RunningJobsRegistry.java   | 32 ++++++++
 .../highavailability/ZookeeperRegistry.java     | 42 +++++++++-
 .../highavailability/nonha/NonHaRegistry.java   | 31 +++++++
 .../runtime/jobmaster/JobManagerRunner.java     | 35 ++++----
 .../minicluster/MiniClusterJobDispatcher.java   | 30 ++++---
 .../FsNegativeRunningJobsRegistryTest.java      | 25 ++++--
 .../highavailability/ZooKeeperRegistryTest.java | 15 +++-
 .../yarn/YarnFlinkApplicationMasterRunner.java  | 19 ++---
 9 files changed, 249 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
index 9d8b226..9e92263 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
@@ -30,14 +30,19 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * This {@link RunningJobsRegistry} tracks the status jobs via marker files,
- * marking finished jobs via marker files.
+ * marking running jobs via running marker files,
+ * marking finished jobs via finished marker files.
  * 
  * <p>The general contract is the following:
  * <ul>
  *     <li>Initially, a marker file does not exist (no one created it, yet), which means
- *         the specific job is assumed to be running</li>
+ *         the specific job is pending</li>
+ *     <li>The first JobManager that granted leadership calls this service to create the running marker file,
+ *         which marks the job as running.</li>
  *     <li>The JobManager that finishes calls this service to create the marker file,
  *         which marks the job as finished.</li>
+ *     <li>If a JobManager gains leadership but see the running marker file,
+ *         it will realize that the job has been scheduled and need reconciling.</li>
  *     <li>If a JobManager gains leadership at some point when shutdown is in progress,
  *         it will see the marker file and realize that the job is finished.</li>
  *     <li>The application framework is expected to clean the file once the application
@@ -52,7 +57,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 
-	private static final String PREFIX = ".job_complete_";
+	private static final String DONE_PREFIX = ".job_complete_";
+
+	private static final String RUNNING_PREFIX = ".job_runing_";
 
 	private final FileSystem fileSystem;
 
@@ -108,21 +115,19 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 	@Override
 	public void setJobRunning(JobID jobID) throws IOException {
 		checkNotNull(jobID, "jobID");
-		final Path filePath = createMarkerFilePath(jobID);
+		final Path filePath = createMarkerFilePath(RUNNING_PREFIX, jobID);
 
-		// delete the marker file, if it exists
-		try {
-			fileSystem.delete(filePath, false);
-		}
-		catch (FileNotFoundException e) {
-			// apparently job was already considered running
+		// create the file
+		// to avoid an exception if the job already exists, set overwrite=true
+		try (FSDataOutputStream out = fileSystem.create(filePath, true)) {
+			out.write(42);
 		}
 	}
 
 	@Override
 	public void setJobFinished(JobID jobID) throws IOException {
 		checkNotNull(jobID, "jobID");
-		final Path filePath = createMarkerFilePath(jobID);
+		final Path filePath = createMarkerFilePath(DONE_PREFIX, jobID);
 
 		// create the file
 		// to avoid an exception if the job already exists, set overwrite=true
@@ -137,17 +142,64 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 
 		// check for the existence of the file
 		try {
-			fileSystem.getFileStatus(createMarkerFilePath(jobID));
-			// file was found --> job is terminated
+			fileSystem.getFileStatus(createMarkerFilePath(RUNNING_PREFIX, jobID));
+			// file was found --> job is running
+			return true;
+		}
+		catch (FileNotFoundException e) {
+			// file does not exist, job is not running
 			return false;
 		}
+	}
+
+	@Override
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
+		checkNotNull(jobID, "jobID");
+
+		// first check for the existence of the complete file
+		try {
+			fileSystem.getFileStatus(createMarkerFilePath(DONE_PREFIX, jobID));
+			// complete file was found --> job is terminated
+			return JobSchedulingStatus.DONE;
+		}
+		catch (FileNotFoundException e) {
+			// file does not exist, job is running or pending
+		}
+		// check for the existence of the running file
+		try {
+			fileSystem.getFileStatus(createMarkerFilePath(RUNNING_PREFIX, jobID));
+			// running file was found --> job is terminated
+			return JobSchedulingStatus.RUNNING;
+		}
+		catch (FileNotFoundException e) {
+			// file does not exist, job is not scheduled
+			return JobSchedulingStatus.PENDING;
+		}
+	}
+
+	@Override
+	public void clearJob(JobID jobID) throws IOException {
+		checkNotNull(jobID, "jobID");
+		final Path runningFilePath = createMarkerFilePath(RUNNING_PREFIX, jobID);
+
+		// delete the running marker file, if it exists
+		try {
+			fileSystem.delete(runningFilePath, false);
+		}
+		catch (FileNotFoundException e) {
+		}
+
+		final Path doneFilePath = createMarkerFilePath(DONE_PREFIX, jobID);
+
+		// delete the finished marker file, if it exists
+		try {
+			fileSystem.delete(doneFilePath, false);
+		}
 		catch (FileNotFoundException e) {
-			// file does not exist, job is still running
-			return true;
 		}
 	}
 
-	private Path createMarkerFilePath(JobID jobId) {
-		return new Path(basePath, PREFIX + jobId.toString());
+	private Path createMarkerFilePath(String prefix, JobID jobId) {
+		return new Path(basePath, prefix + jobId.toString());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
index e7c131c..020f2ca 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
@@ -33,6 +33,17 @@ import java.io.IOException;
  */
 public interface RunningJobsRegistry {
 
+	public enum JobSchedulingStatus {
+		/** Job has not been scheduled */
+		PENDING,
+
+		/** Job has been scheduled */
+		RUNNING,
+
+		/** Job has been finished */
+		DONE;
+	}
+
 	/**
 	 * Marks a job as running.
 	 * 
@@ -63,4 +74,25 @@ public interface RunningJobsRegistry {
 	 *                     failed and could not be retried.
 	 */
 	boolean isJobRunning(JobID jobID) throws IOException;
+
+	/**
+	 * Get the scheduing status of a job.
+	 *
+	 * @param jobID The id of the job to check.
+	 * @return The job scheduling status.
+	 * 
+	 * @throws IOException Thrown when the communication with the highly-available storage or registry
+	 *                     failed and could not be retried.
+	 */
+	JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException;
+
+	/**
+	 * Clear job state form the registry, usually called after job finish
+	 *
+	 * @param jobID The id of the job to check.
+	 * 
+	 * @throws IOException Thrown when the communication with the highly-available storage or registry
+	 *                     failed and could not be retried.
+	 */
+	void clearJob(JobID jobID) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
index c0621af..31a4535 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.zookeeper.data.Stat;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -55,7 +56,7 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		try {
 			String zkPath = runningJobPath + jobID.toString();
 			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-			this.client.setData().forPath(zkPath);
+			this.client.setData().forPath(zkPath, JobSchedulingStatus.RUNNING.name().getBytes(Charset.forName("utf-8")));
 		}
 		catch (Exception e) {
 			throw new IOException("Set running state to zk fail for job " + jobID.toString(), e);
@@ -69,7 +70,7 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		try {
 			String zkPath = runningJobPath + jobID.toString();
 			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-			this.client.delete().forPath(zkPath);
+			this.client.setData().forPath(zkPath, JobSchedulingStatus.DONE.name().getBytes(Charset.forName("utf-8")));
 		}
 		catch (Exception e) {
 			throw new IOException("Set finished state to zk fail for job " + jobID.toString(), e);
@@ -83,7 +84,10 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		try {
 			Stat stat = client.checkExists().forPath(runningJobPath + jobID.toString());
 			if (stat != null) {
-				return true;
+				byte[] data = client.getData().forPath(runningJobPath + jobID.toString());
+				if (JobSchedulingStatus.RUNNING.name().equals(new String(data))) {
+					return true;
+				}
 			}
 			return false;
 		}
@@ -91,4 +95,36 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 			throw new IOException("Get running state from zk fail for job " + jobID.toString(), e);
 		}
 	}
+
+	@Override
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			Stat stat = client.checkExists().forPath(runningJobPath + jobID.toString());
+			if (stat != null) {
+				byte[] data = client.getData().forPath(runningJobPath + jobID.toString());
+				return JobSchedulingStatus.valueOf(new String(data));
+			}
+			return JobSchedulingStatus.PENDING;
+		}
+		catch (Exception e) {
+			throw new IOException("Get finished state from zk fail for job " + jobID.toString(), e);
+		}
+	}
+
+	@Override
+	public void clearJob(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			String zkPath = runningJobPath + jobID.toString();
+			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
+			this.client.delete().forPath(zkPath);
+		}
+		catch (Exception e) {
+			throw new IOException("Clear job state from zk fail for " + jobID.toString(), e);
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
index 85dd711..e331212 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.highavailability.nonha;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 
 import java.util.HashSet;
 
@@ -33,12 +34,16 @@ public class NonHaRegistry implements RunningJobsRegistry {
 	/** The currently running jobs */
 	private final HashSet<JobID> running = new HashSet<>();
 
+	/** The currently finished jobs */
+	private final HashSet<JobID> finished = new HashSet<>();
+
 	@Override
 	public void setJobRunning(JobID jobID) {
 		checkNotNull(jobID);
 
 		synchronized (running) {
 			running.add(jobID);
+			finished.remove(jobID);
 		}
 	}
 
@@ -48,6 +53,7 @@ public class NonHaRegistry implements RunningJobsRegistry {
 
 		synchronized (running) {
 			running.remove(jobID);
+			finished.add(jobID);
 		}
 	}
 
@@ -59,4 +65,29 @@ public class NonHaRegistry implements RunningJobsRegistry {
 			return running.contains(jobID);
 		}
 	}
+
+	@Override
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (running) {
+			if (finished.contains(jobID)) {
+				return JobSchedulingStatus.DONE;
+			} else if (running.contains(jobID)) {
+				return JobSchedulingStatus.RUNNING;
+			} else {
+				return JobSchedulingStatus.PENDING;
+			}
+		}
+	}
+
+	@Override
+	public void clearJob(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (running) {
+			running.remove(jobID);
+			finished.remove(jobID);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
index 9d8e004..6bebd90 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
@@ -359,29 +360,35 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
 			// it's okay that job manager wait for the operation complete
 			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
 
-			boolean jobRunning;
+			JobSchedulingStatus schedulingStatus = JobSchedulingStatus.PENDING;
 			try {
-				jobRunning = runningJobsRegistry.isJobRunning(jobGraph.getJobID());
+				schedulingStatus = runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID());
+				if (schedulingStatus.equals(JobSchedulingStatus.DONE)) {
+					log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID());
+					jobFinishedByOther();
+					return;
+				}
 			} catch (Throwable t) {
-				log.error("Could not access status (running/finished) of job {}. " +
-						"Falling back to assumption that job is running and attempting recovery...",
-						jobGraph.getJobID(), t);
-				jobRunning = true;
+				log.error("Could not access status (running/finished) of job {}. ",	jobGraph.getJobID(), t);
+				onFatalError(t);
+				return;
 			}
 
 			// Double check the leadership after we confirm that, there is a small chance that multiple
 			// job managers schedule the same job after if they try to recover at the same time.
 			// This will eventually be noticed, but can not be ruled out from the beginning.
 			if (leaderElectionService.hasLeadership()) {
-				if (jobRunning) {
-					try {
-						jobManager.start(leaderSessionID);
-					} catch (Exception e) {
-						onFatalError(new Exception("Could not start the job manager.", e));
+				try {
+					// Now set the running status is after getting leader ship and 
+					// set finished status after job in terminated status.
+					// So if finding the job is running, it means someone has already run the job, need recover.
+					if (schedulingStatus.equals(JobSchedulingStatus.PENDING)) {
+						runningJobsRegistry.setJobRunning(jobGraph.getJobID());
 					}
-				} else {
-					log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
-					jobFinishedByOther();
+
+					jobManager.start(leaderSessionID);
+				} catch (Exception e) {
+					onFatalError(new Exception("Could not start the job manager.", e));
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
index 7fffaee..9178684 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -182,7 +182,7 @@ public class MiniClusterJobDispatcher {
 			checkState(!shutdown, "mini cluster is shut down");
 			checkState(runners == null, "mini cluster can only execute one job at a time");
 
-			DetachedFinalizer finalizer = new DetachedFinalizer(numJobManagers);
+			DetachedFinalizer finalizer = new DetachedFinalizer(job.getJobID(), numJobManagers);
 
 			this.runners = startJobRunners(job, finalizer, finalizer);
 		}
@@ -217,6 +217,7 @@ public class MiniClusterJobDispatcher {
 		finally {
 			// always clear the status for the next job
 			runners = null;
+			clearJobRunningState(job.getJobID());
 		}
 	}
 
@@ -227,16 +228,6 @@ public class MiniClusterJobDispatcher {
 
 		LOG.info("Starting {} JobMaster(s) for job {} ({})", numJobManagers, job.getName(), job.getJobID());
 
-		// we first need to mark the job as running in the HA services, so that the
-		// JobManager leader will recognize that it as work to do
-		try {
-			haServices.getRunningJobsRegistry().setJobRunning(job.getJobID());
-		}
-		catch (Throwable t) {
-			throw new JobExecutionException(job.getJobID(),
-					"Could not register the job at the high-availability services", t);
-		}
-
 		// start all JobManagers
 		JobManagerRunner[] runners = new JobManagerRunner[numJobManagers];
 		for (int i = 0; i < numJobManagers; i++) {
@@ -273,6 +264,17 @@ public class MiniClusterJobDispatcher {
 		return runners;
 	}
 
+	private void clearJobRunningState(JobID jobID) {
+		// we mark the job as finished in the HA services, so need
+		// to remove the data after job finished
+		try {
+			haServices.getRunningJobsRegistry().clearJob(jobID);
+		}
+		catch (Throwable t) {
+			LOG.warn("Could not clear the job {} at the high-availability services", jobID.toString(), t);
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  test methods to simulate job master failures
 	// ------------------------------------------------------------------------
@@ -298,9 +300,12 @@ public class MiniClusterJobDispatcher {
 	 */
 	private class DetachedFinalizer implements OnCompletionActions, FatalErrorHandler {
 
+		private final JobID jobID;
+
 		private final AtomicInteger numJobManagersToWaitFor;
 
-		private DetachedFinalizer(int numJobManagersToWaitFor) {
+		private DetachedFinalizer(JobID jobID, int numJobManagersToWaitFor) {
+			this.jobID = jobID;
 			this.numJobManagersToWaitFor = new AtomicInteger(numJobManagersToWaitFor);
 		}
 
@@ -327,6 +332,7 @@ public class MiniClusterJobDispatcher {
 		private void decrementCheckAndCleanup() {
 			if (numJobManagersToWaitFor.decrementAndGet() == 0) {
 				MiniClusterJobDispatcher.this.runners = null;
+				MiniClusterJobDispatcher.this.clearJobRunningState(jobID);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
index f1ece0e..bbafcf0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Rule;
@@ -28,6 +29,7 @@ import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -45,20 +47,23 @@ public class FsNegativeRunningJobsRegistryTest extends TestLogger {
 
 		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri));
 
-		// initially, without any call, the job is considered running
-		assertTrue(registry.isJobRunning(jid));
+		// initially, without any call, the job is pending
+		assertFalse(registry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.PENDING);
 
-		// repeated setting should not affect the status
+		// after set running, the job is running
 		registry.setJobRunning(jid);
 		assertTrue(registry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.RUNNING);
 
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
-		assertFalse(registry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
 
 		// another registry should pick this up
 		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
-		assertFalse(otherRegistry.isJobRunning(jid));
+		assertTrue(otherRegistry.isJobRunning(jid));
+		assertEquals(otherRegistry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
 	}
 
 	@Test
@@ -73,13 +78,21 @@ public class FsNegativeRunningJobsRegistryTest extends TestLogger {
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
 		assertFalse(registry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
 
-		// set the job to back to running and validate
+		// set the job to running does not overwrite the finished status
 		registry.setJobRunning(jid);
 		assertTrue(registry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
 
 		// another registry should pick this up
 		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
 		assertTrue(otherRegistry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
+
+		// clear the running and finished marker, it will be pending
+		otherRegistry.clearJob(jid);
+		assertFalse(otherRegistry.isJobRunning(jid));
+		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.PENDING);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
index 72982c8..8c91898 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
@@ -22,6 +22,7 @@ import org.apache.curator.test.TestingServer;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
@@ -29,7 +30,9 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 public class ZooKeeperRegistryTest extends TestLogger {
 	private TestingServer testingServer;
@@ -61,14 +64,20 @@ public class ZooKeeperRegistryTest extends TestLogger {
 
 		try {
 			JobID jobID = JobID.generate();
-			assertTrue(!zkRegistry.isJobRunning(jobID));
+			assertFalse(zkRegistry.isJobRunning(jobID));
+			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.PENDING);
 
 			zkRegistry.setJobRunning(jobID);
 			assertTrue(zkRegistry.isJobRunning(jobID));
+			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.RUNNING);
 
 			zkRegistry.setJobFinished(jobID);
-			assertTrue(!zkRegistry.isJobRunning(jobID));
+			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.DONE);
+			assertFalse(zkRegistry.isJobRunning(jobID));
 
+			zkRegistry.clearJob(jobID);
+			assertFalse(zkRegistry.isJobRunning(jobID));
+			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.PENDING);
 		} finally {
 			if (zkHaService != null) {
 				zkHaService.close();

http://git-wip-us.apache.org/repos/asf/flink/blob/e7011d78/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index 257212b..e2aa6ec 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -24,7 +24,6 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -206,16 +205,6 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 		//TODO: generate the job graph from user's jar
 		jobGraph = loadJobGraph(config);
 
-		// we first need to mark the job as running in the HA services, so that the
-		// JobManager leader will recognize that it as work to do
-		try {
-			haServices.getRunningJobsRegistry().setJobRunning(jobGraph.getJobID());
-		}
-		catch (Throwable t) {
-			throw new JobExecutionException(jobGraph.getJobID(),
-					"Could not register the job at the high-availability services", t);
-		}
-
 		// now the JobManagerRunner
 		return new JobManagerRunner(
 				jobGraph, config,
@@ -226,6 +215,14 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 	}
 
 	protected void shutdown(ApplicationStatus status, String msg) {
+		// Need to clear the job state in the HA services before shutdown
+		try {
+			haServices.getRunningJobsRegistry().clearJob(jobGraph.getJobID());
+		}
+		catch (Throwable t) {
+			LOG.warn("Could not clear the job at the high-availability services", t);
+		}
+
 		synchronized (lock) {
 			if (jobManagerRunner != null) {
 				try {


[11/11] flink git commit: [hotfix] [tests] Fix test instability in SavepointITCase

Posted by se...@apache.org.
[hotfix] [tests] Fix test instability in SavepointITCase


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

Branch: refs/heads/master
Commit: 3086af534cfcea0ccb39d3058c172983cc17dabe
Parents: c477d87
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Feb 28 19:16:48 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 19:16:48 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/test/checkpointing/SavepointITCase.java   | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3086af53/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index ee371dd..ed45807 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -61,6 +61,7 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestSavepoint;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ResponseSavepoint;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.ResponseSubmitTaskListener;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
@@ -180,6 +181,11 @@ public class SavepointITCase extends TestLogger {
 
 			LOG.info("Waiting for some progress.");
 
+			// wait for the JobManager to be ready
+			Future<Object> allRunning = jobManager.ask(new WaitForAllVerticesToBeRunning(jobId), deadline.timeLeft());
+			Await.ready(allRunning, deadline.timeLeft());
+
+			// wait for the Tasks to be ready
 			StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 			LOG.info("Triggering a savepoint.");


[06/11] flink git commit: [FLINK-5897] [checkpoints] Make checkpoint externalization not depend strictly on FileSystems

Posted by se...@apache.org.
[FLINK-5897] [checkpoints] Make checkpoint externalization not depend strictly on FileSystems

That is the first step towards checkpoints that can be externalized to other stores as well,
like k/v stores and databases, if supported by the state backend.


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

Branch: refs/heads/master
Commit: 5b7f21d891b410ca0046efdaf12caf5e73deadf4
Parents: 9912de2
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Feb 22 22:18:50 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |   4 +-
 .../checkpoint/CheckpointCoordinator.java       |  20 ++-
 .../runtime/checkpoint/CompletedCheckpoint.java | 135 +++++++++++++++----
 .../checkpoint/CompletedCheckpointStore.java    |   9 ++
 .../runtime/checkpoint/PendingCheckpoint.java   | 112 ++++++++++-----
 .../StandaloneCompletedCheckpointStore.java     |   4 +
 .../ZooKeeperCompletedCheckpointStore.java      |   5 +
 .../checkpoint/savepoint/SavepointLoader.java   |  19 ++-
 .../checkpoint/savepoint/SavepointStore.java    |  93 ++++++++++---
 .../apache/flink/runtime/state/StateUtil.java   |  17 +--
 .../flink/runtime/jobmanager/JobManager.scala   |  14 +-
 .../CheckpointCoordinatorFailureTest.java       |   5 +
 .../CompletedCheckpointStoreTest.java           |   2 +-
 .../checkpoint/CompletedCheckpointTest.java     |  17 ++-
 .../checkpoint/PendingCheckpointTest.java       |  25 ++--
 .../jobmanager/JobManagerHARecoveryTest.java    |   4 +
 .../runtime/jobmanager/JobManagerITCase.scala   |   2 +-
 .../JobManagerHACheckpointRecoveryITCase.java   |   2 +-
 18 files changed, 365 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/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 fea25ff..7167a0b 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
@@ -257,7 +257,7 @@ public final class ExceptionUtils {
 			throw (Error) t;
 		}
 		else {
-			throw new IOException(t);
+			throw new IOException(t.getMessage(), t);
 		}
 	}
 
@@ -268,7 +268,7 @@ public final class ExceptionUtils {
 	 * @param searchType the type of exception to search for in the chain.
 	 * @return True, if the searched type is nested in the throwable, false otherwise.
 	 */
-	public static boolean containsThrowable(Throwable throwable, Class searchType) {
+	public static boolean containsThrowable(Throwable throwable, Class<?> searchType) {
 		if (throwable == null || searchType == null) {
 			return false;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index c1c65b5..6da6f7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.TaskStateHandles;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -758,13 +759,19 @@ public class CheckpointCoordinator {
 		CompletedCheckpoint completedCheckpoint = null;
 
 		try {
-			completedCheckpoint = pendingCheckpoint.finalizeCheckpoint();
+			// externalize the checkpoint if required
+			if (pendingCheckpoint.getProps().externalizeCheckpoint()) {
+				completedCheckpoint = pendingCheckpoint.finalizeCheckpointExternalized();
+			} else {
+				completedCheckpoint = pendingCheckpoint.finalizeCheckpointNonExternalized();
+			}
 
 			completedCheckpointStore.addCheckpoint(completedCheckpoint);
 
 			rememberRecentCheckpointId(checkpointId);
 			dropSubsumedCheckpoints(checkpointId);
-		} catch (Exception exception) {
+		}
+		catch (Exception exception) {
 			// abort the current pending checkpoint if it has not been discarded yet
 			if (!pendingCheckpoint.isDiscarded()) {
 				pendingCheckpoint.abortError(exception);
@@ -779,8 +786,8 @@ public class CheckpointCoordinator {
 					public void run() {
 						try {
 							cc.discard();
-						} catch (Exception nestedException) {
-							LOG.warn("Could not properly discard completed checkpoint {}.", cc.getCheckpointID(), nestedException);
+						} catch (Throwable t) {
+							LOG.warn("Could not properly discard completed checkpoint {}.", cc.getCheckpointID(), t);
 						}
 					}
 				});
@@ -808,11 +815,12 @@ public class CheckpointCoordinator {
 				builder.append(", ");
 			}
 			// Remove last two chars ", "
-			builder.delete(builder.length() - 2, builder.length());
+			builder.setLength(builder.length() - 2);
 
 			LOG.debug(builder.toString());
 		}
 
+		// send the "notify complete" call to all vertices
 		final long timestamp = completedCheckpoint.getTimestamp();
 
 		for (ExecutionVertex ev : tasksToCommitTo) {
@@ -934,7 +942,7 @@ public class CheckpointCoordinator {
 					latest.getCheckpointID(),
 					latest.getProperties(),
 					restoreTimestamp,
-					latest.getExternalPath());
+					latest.getExternalPointer());
 
 				statsTracker.reportRestoredCheckpoint(restored);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
index db86484..17ce4d5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
@@ -18,11 +18,14 @@
 
 package org.apache.flink.runtime.checkpoint;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats.DiscardCallback;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.ExceptionUtils;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,8 +38,36 @@ import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * A successful checkpoint describes a checkpoint after all required tasks acknowledged it (with their state)
- * and that is considered completed.
+ * A CompletedCheckpoint describes a checkpoint after all required tasks acknowledged it (with their state)
+ * and that is considered successful. The CompletedCheckpoint class contains all the metadata of the
+ * checkpoint, i.e., checkpoint ID, timestamps, and the handles to all states that are part of the
+ * checkpoint.
+ * 
+ * <h2>Size the CompletedCheckpoint Instances</h2>
+ * 
+ * In most cases, the CompletedCheckpoint objects are very small, because the handles to the checkpoint
+ * states are only pointers (such as file paths). However, the some state backend implementations may
+ * choose to store some payload data directly with the metadata (for example to avoid many small files).
+ * If those thresholds are increased to large values, the memory consumption of the CompletedCheckpoint
+ * objects can be significant.
+ * 
+ * <h2>Externalized Metadata</h2>
+ * 
+ * The metadata of the CompletedCheckpoint is optionally also persisted in an external storage
+ * system. In that case, the checkpoint is called <i>externalized</i>.
+ * 
+ * <p>Externalized checkpoints have an external pointer, which points to the metadata. For example
+ * when externalizing to a file system, that pointer is the file path to the checkpoint's folder
+ * or the metadata file. For a state backend that stores metadata in database tables, the pointer
+ * could be the table name and row key. The pointer is encoded as a String.
+ * 
+ * <h2>Externalized Metadata and High-availability</h2>
+ * 
+ * For high availability setups, the checkpoint metadata must be stored persistent and available
+ * as well. The high-availability services that stores the checkpoint ground-truth (meaning what are
+ * the latest completed checkpoints in what order) often rely on checkpoints being externalized. That
+ * way, those services only store pointers to the externalized metadata, rather than the complete
+ * metadata itself (for example ZooKeeper's ZNode payload should ideally be less than megabytes).
  */
 public class CompletedCheckpoint implements Serializable {
 
@@ -44,8 +75,12 @@ public class CompletedCheckpoint implements Serializable {
 
 	private static final long serialVersionUID = -8360248179615702014L;
 
+	// ------------------------------------------------------------------------
+
+	/** The ID of the job that the checkpoint belongs to */
 	private final JobID job;
 
+	/** The ID (logical timestamp) of the checkpoint */
 	private final long checkpointID;
 
 	/** The timestamp when the checkpoint was triggered. */
@@ -60,23 +95,41 @@ public class CompletedCheckpoint implements Serializable {
 	/** Properties for this checkpoint. */
 	private final CheckpointProperties props;
 
-	/** External path if persisted checkpoint; <code>null</code> otherwise. */
-	private final String externalPath;
+	/** The state handle to the externalized meta data, if the metadata has been externalized */
+	@Nullable
+	private final StreamStateHandle externalizedMetadata;
+
+	/** External pointer to the completed checkpoint (for example file path) if externalized; null otherwise. */
+	@Nullable
+	private final String externalPointer;
 
 	/** Optional stats tracker callback for discard. */
 	@Nullable
-	private transient CompletedCheckpointStats.DiscardCallback discardCallback;
+	private transient volatile DiscardCallback discardCallback;
 
 	// ------------------------------------------------------------------------
 
-	public CompletedCheckpoint(
+	@VisibleForTesting
+	CompletedCheckpoint(
 			JobID job,
 			long checkpointID,
 			long timestamp,
 			long completionTimestamp,
 			Map<JobVertexID, TaskState> taskStates) {
 
-		this(job, checkpointID, timestamp, completionTimestamp, taskStates, CheckpointProperties.forStandardCheckpoint(), null);
+		this(job, checkpointID, timestamp, completionTimestamp, taskStates,
+				CheckpointProperties.forStandardCheckpoint());
+	}
+
+	public CompletedCheckpoint(
+			JobID job,
+			long checkpointID,
+			long timestamp,
+			long completionTimestamp,
+			Map<JobVertexID, TaskState> taskStates,
+			CheckpointProperties props) {
+
+		this(job, checkpointID, timestamp, completionTimestamp, taskStates, props, null, null);
 	}
 
 	public CompletedCheckpoint(
@@ -86,24 +139,27 @@ public class CompletedCheckpoint implements Serializable {
 			long completionTimestamp,
 			Map<JobVertexID, TaskState> taskStates,
 			CheckpointProperties props,
-			String externalPath) {
+			@Nullable StreamStateHandle externalizedMetadata,
+			@Nullable String externalPointer) {
 
 		checkArgument(checkpointID >= 0);
 		checkArgument(timestamp >= 0);
 		checkArgument(completionTimestamp >= 0);
 
+		checkArgument((externalPointer == null) == (externalizedMetadata == null),
+				"external pointer without externalized metadata must be both null or both non-null");
+
+		checkArgument(!props.externalizeCheckpoint() || externalPointer != null, 
+			"Checkpoint properties require externalized checkpoint, but checkpoint is not externalized");
+
 		this.job = checkNotNull(job);
 		this.checkpointID = checkpointID;
 		this.timestamp = timestamp;
 		this.duration = completionTimestamp - timestamp;
 		this.taskStates = checkNotNull(taskStates);
 		this.props = checkNotNull(props);
-		this.externalPath = externalPath;
-
-		if (props.externalizeCheckpoint() && externalPath == null) {
-			throw new NullPointerException("Checkpoint properties say that the checkpoint " +
-					"should have been persisted, but missing external path.");
-		}
+		this.externalizedMetadata = externalizedMetadata;
+		this.externalPointer = externalPointer;
 	}
 
 	// ------------------------------------------------------------------------
@@ -146,10 +202,9 @@ public class CompletedCheckpoint implements Serializable {
 			discard();
 			return true;
 		} else {
-			if (externalPath != null) {
+			if (externalPointer != null) {
 				LOG.info("Persistent checkpoint with ID {} at '{}' not discarded.",
-						checkpointID,
-						externalPath);
+						checkpointID, externalPointer);
 			}
 
 			return false;
@@ -158,14 +213,36 @@ public class CompletedCheckpoint implements Serializable {
 
 	void discard() throws Exception {
 		try {
-			if (externalPath != null) {
-				SavepointStore.removeSavepointFile(externalPath);
+			// collect exceptions and continue cleanup
+			Exception exception = null;
+
+			// drop the metadata, if we have some
+			if (externalizedMetadata != null) {
+				try {
+					externalizedMetadata.discardState();
+				}
+				catch (Exception e) {
+					exception = e;
+				}
 			}
 
-			StateUtil.bestEffortDiscardAllStateObjects(taskStates.values());
-		} finally {
+			// drop the actual state
+			try {
+				StateUtil.bestEffortDiscardAllStateObjects(taskStates.values());
+			}
+			catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw exception;
+			}
+		}
+		finally {
 			taskStates.clear();
 
+			// to be null-pointer safe, copy reference to stack
+			DiscardCallback discardCallback = this.discardCallback;
 			if (discardCallback != null) {
 				discardCallback.notifyDiscardedCheckpoint();
 			}
@@ -190,8 +267,18 @@ public class CompletedCheckpoint implements Serializable {
 		return taskStates.get(jobVertexID);
 	}
 
-	public String getExternalPath() {
-		return externalPath;
+	public boolean isExternalized() {
+		return externalizedMetadata != null;
+	}
+
+	@Nullable
+	public StreamStateHandle getExternalizedMetadata() {
+		return externalizedMetadata;
+	}
+
+	@Nullable
+	public String getExternalPointer() {
+		return externalPointer;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
index d2c0f6c..e91e038 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
@@ -72,4 +72,13 @@ public interface CompletedCheckpointStore {
 	 */
 	int getNumberOfRetainedCheckpoints();
 
+	/**
+	 * This method returns whether the completed checkpoint store requires checkpoints to be
+	 * externalized. Externalized checkpoints have their meta data persisted, which the checkpoint
+	 * store can exploit (for example by simply pointing the persisted metadata).
+	 * 
+	 * @return True, if the store requires that checkpoints are externalized before being added, false
+	 *         if the store stores the metadata itself.
+	 */
+	boolean requiresExternalizedCheckpoints();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index 908ff7f..2c392b8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.checkpoint;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -28,6 +29,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Executor;
 import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.checkpoint.savepoint.Savepoint;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
@@ -41,7 +44,10 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateUtil;
 import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,7 +80,7 @@ public class PendingCheckpoint {
 
 	/**
 	 * The checkpoint properties. If the checkpoint should be persisted
-	 * externally, it happens in {@link #finalizeCheckpoint()}.
+	 * externally, it happens in {@link #finalizeCheckpointExternalized()}.
 	 */
 	private final CheckpointProperties props;
 
@@ -203,46 +209,80 @@ public class PendingCheckpoint {
 		return onCompletionPromise;
 	}
 
-	public CompletedCheckpoint finalizeCheckpoint() {
+	public CompletedCheckpoint finalizeCheckpointExternalized() throws IOException {
 		synchronized (lock) {
-			Preconditions.checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet.");
-
-			// Persist if required
-			String externalPath = null;
-			if (props.externalizeCheckpoint()) {
-				try {
-					Savepoint savepoint = new SavepointV1(checkpointId, taskStates.values());
-					externalPath = SavepointStore.storeSavepoint(
-							targetDirectory,
-							savepoint
-					);
-				} catch (IOException e) {
-					LOG.error("Failed to persist checkpoint {}.",checkpointId, e);
-				}
-			}
+			checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet.");
 
-			CompletedCheckpoint completed = new CompletedCheckpoint(
-					jobId,
-					checkpointId,
-					checkpointTimestamp,
-					System.currentTimeMillis(),
-					new HashMap<>(taskStates),
-					props,
-					externalPath);
+			// make sure we fulfill the promise with an exception if something fails
+			try {
+				// externalize the metadata
+				final Savepoint savepoint = new SavepointV1(checkpointId, taskStates.values());
 
-			onCompletionPromise.complete(completed);
+				// TEMP FIX - The savepoint store is strictly typed to file systems currently
+				//            but the checkpoints think more generic. we need to work with file handles
+				//            here until the savepoint serializer accepts a generic stream factory
 
-			if (statsCallback != null) {
-				// Finalize the statsCallback and give the completed checkpoint a
-				// callback for discards.
-				CompletedCheckpointStats.DiscardCallback discardCallback = statsCallback.reportCompletedCheckpoint(externalPath);
-				completed.setDiscardCallback(discardCallback);
+				final FileStateHandle metadataHandle = SavepointStore.storeSavepointToHandle(targetDirectory, savepoint);
+				final String externalPointer = metadataHandle.getFilePath().getParent().toString();
+
+				return finalizeInternal(metadataHandle, externalPointer);
+			}
+			catch (Throwable t) {
+				onCompletionPromise.completeExceptionally(t);
+				ExceptionUtils.rethrowIOException(t);
+				return null; // silence the compiler
+			}
+		}
+	}
+
+	public CompletedCheckpoint finalizeCheckpointNonExternalized() {
+		synchronized (lock) {
+			checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet.");
+
+			// make sure we fulfill the promise with an exception if something fails
+			try {
+				// finalize without external metadata
+				return finalizeInternal(null, null);
 			}
+			catch (Throwable t) {
+				onCompletionPromise.completeExceptionally(t);
+				ExceptionUtils.rethrow(t);
+				return null; // silence the compiler
+			}
+		}
+	}
 
-			dispose(false);
+	@GuardedBy("lock")
+	private CompletedCheckpoint finalizeInternal(
+			@Nullable StreamStateHandle externalMetadata,
+			@Nullable String externalPointer) {
 
-			return completed;
+		assert(Thread.holdsLock(lock));
+
+		CompletedCheckpoint completed = new CompletedCheckpoint(
+				jobId,
+				checkpointId,
+				checkpointTimestamp,
+				System.currentTimeMillis(),
+				new HashMap<>(taskStates),
+				props,
+				externalMetadata,
+				externalPointer);
+
+		onCompletionPromise.complete(completed);
+
+		if (statsCallback != null) {
+			// Finalize the statsCallback and give the completed checkpoint a
+			// callback for discards.
+			CompletedCheckpointStats.DiscardCallback discardCallback = 
+					statsCallback.reportCompletedCheckpoint(externalPointer);
+			completed.setDiscardCallback(discardCallback);
 		}
+
+		// mark this pending checkpoint as disposed, but do NOT drop the state
+		dispose(false);
+
+		return completed;
 	}
 
 	/**
@@ -411,9 +451,9 @@ public class PendingCheckpoint {
 						public void run() {
 							try {
 								StateUtil.bestEffortDiscardAllStateObjects(taskStates.values());
-							} catch (Exception e) {
-								LOG.warn("Could not properly dispose the pending checkpoint " +
-									"{} of job {}.", checkpointId, jobId, e);
+							} catch (Throwable t) {
+								LOG.warn("Could not properly dispose the pending checkpoint {} of job {}.", 
+										checkpointId, jobId, t);
 							} finally {
 								taskStates.clear();
 							}

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
index 082bca9..a0248b2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
@@ -96,4 +96,8 @@ public class StandaloneCompletedCheckpointStore implements CompletedCheckpointSt
 		}
 	}
 
+	@Override
+	public boolean requiresExternalizedCheckpoints() {
+		return false;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
index fdd0d40..4b03cea 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
@@ -125,6 +125,11 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto
 		LOG.info("Initialized in '{}'.", checkpointsPath);
 	}
 
+	@Override
+	public boolean requiresExternalizedCheckpoints() {
+		return true;
+	}
+
 	/**
 	 * Gets the latest checkpoint from ZooKeeper and removes all others.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java
index 950a9a0..60f0287 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java
@@ -19,11 +19,13 @@
 package org.apache.flink.runtime.checkpoint.savepoint;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.checkpoint.CheckpointProperties;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
 import org.apache.flink.runtime.checkpoint.TaskState;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,22 +48,27 @@ public class SavepointLoader {
 	 * @param jobId          The JobID of the job to load the savepoint for.
 	 * @param tasks          Tasks that will possibly be reset
 	 * @param savepointPath  The path of the savepoint to rollback to
-	 * @param userClassLoader The user code classloader
+	 * @param classLoader    The class loader to resolve serialized classes in legacy savepoint versions.
 	 * @param allowNonRestoredState Allow to skip checkpoint state that cannot be mapped
 	 * to any job vertex in tasks.
 	 *
 	 * @throws IllegalStateException If mismatch between program and savepoint state
-	 * @throws Exception             If savepoint store failure
+	 * @throws IOException             If savepoint store failure
 	 */
 	public static CompletedCheckpoint loadAndValidateSavepoint(
 			JobID jobId,
 			Map<JobVertexID, ExecutionJobVertex> tasks,
 			String savepointPath,
-			ClassLoader userClassLoader,
+			ClassLoader classLoader,
 			boolean allowNonRestoredState) throws IOException {
 
 		// (1) load the savepoint
-		Savepoint savepoint = SavepointStore.loadSavepoint(savepointPath, userClassLoader);
+		final Tuple2<Savepoint, StreamStateHandle> savepointAndHandle = 
+				SavepointStore.loadSavepointWithHandle(savepointPath, classLoader);
+
+		final Savepoint savepoint = savepointAndHandle.f0;
+		final StreamStateHandle metadataHandle = savepointAndHandle.f1;
+
 		final Map<JobVertexID, TaskState> taskStates = new HashMap<>(savepoint.getTaskStates().size());
 
 		boolean expandedToLegacyIds = false;
@@ -114,10 +121,12 @@ public class SavepointLoader {
 
 		// (3) convert to checkpoint so the system can fall back to it
 		CheckpointProperties props = CheckpointProperties.forStandardSavepoint();
-		return new CompletedCheckpoint(jobId, savepoint.getCheckpointId(), 0L, 0L, taskStates, props, savepointPath);
+		return new CompletedCheckpoint(jobId, savepoint.getCheckpointId(), 0L, 0L,
+				taskStates, props, metadataHandle, savepointPath);
 	}
 
 	// ------------------------------------------------------------------------
 
+	/** This class is not meant to be instantiated */
 	private SavepointLoader() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
index 95370a5..5c8ac6b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
@@ -19,12 +19,15 @@
 package org.apache.flink.runtime.checkpoint.savepoint;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.Preconditions;
 
@@ -118,6 +121,28 @@ public class SavepointStore {
 	 * @throws IOException Failures during store are forwarded
 	 */
 	public static <T extends Savepoint> String storeSavepoint(String directory, T savepoint) throws IOException {
+		// write and create the file handle
+		FileStateHandle metadataFileHandle = storeSavepointToHandle(directory, savepoint);
+
+		// we return the savepoint directory path here!
+		// The directory path also works to resume from and is more elegant than the direct
+		// metadata file pointer
+		return metadataFileHandle.getFilePath().getParent().toString();
+	}
+
+	/**
+	 * Stores the savepoint metadata file to a state handle.
+	 *
+	 * @param directory Target directory to store savepoint in
+	 * @param savepoint Savepoint to be stored
+	 *                     
+	 * @return State handle to the checkpoint metadata
+	 * @throws IOException Failures during store are forwarded
+	 */
+	public static <T extends Savepoint> FileStateHandle storeSavepointToHandle(
+			String directory,
+			T savepoint) throws IOException {
+
 		checkNotNull(directory, "Target directory");
 		checkNotNull(savepoint, "Savepoint");
 
@@ -127,10 +152,9 @@ public class SavepointStore {
 		final FileSystem fs = FileSystem.get(basePath.toUri());
 
 		boolean success = false;
-		try (FSDataOutputStream fdos = fs.create(metadataFilePath, WriteMode.NO_OVERWRITE); 
+		try (FSDataOutputStream fdos = fs.create(metadataFilePath, WriteMode.NO_OVERWRITE);
 				DataOutputStream dos = new DataOutputStream(fdos))
 		{
-
 			// Write header
 			dos.writeInt(MAGIC_NUMBER);
 			dos.writeInt(savepoint.getVersion());
@@ -138,7 +162,13 @@ public class SavepointStore {
 			// Write savepoint
 			SavepointSerializer<T> serializer = SavepointSerializers.getSerializer(savepoint);
 			serializer.serialize(savepoint, dos);
+
+			// construct result handle
+			FileStateHandle handle = new FileStateHandle(metadataFilePath, dos.size());
+
+			// all good!
 			success = true;
+			return handle;
 		}
 		finally {
 			if (!success && fs.exists(metadataFilePath)) {
@@ -147,22 +177,37 @@ public class SavepointStore {
 				}
 			}
 		}
-
-		// we return the savepoint directory path here!
-		// The directory path also works to resume from and is more elegant than the direct
-		// metadata file pointer
-		return basePath.toString();
 	}
 
 	/**
 	 * Loads the savepoint at the specified path.
 	 *
 	 * @param savepointFileOrDirectory Path to the parent savepoint directory or the meta data file.
+	 * @param classLoader The class loader used to resolve serialized classes from legacy savepoint formats.
 	 * @return The loaded savepoint
+	 * 
 	 * @throws IOException Failures during load are forwarded
 	 */
-	public static Savepoint loadSavepoint(String savepointFileOrDirectory, ClassLoader userClassLoader) throws IOException {
-		Preconditions.checkNotNull(savepointFileOrDirectory, "Path");
+	public static Savepoint loadSavepoint(String savepointFileOrDirectory, ClassLoader classLoader) throws IOException {
+		return loadSavepointWithHandle(savepointFileOrDirectory, classLoader).f0;
+	}
+
+	/**
+	 * Loads the savepoint at the specified path. This methods returns the savepoint, as well as the
+	 * handle to the metadata.
+	 *
+	 * @param savepointFileOrDirectory Path to the parent savepoint directory or the meta data file.
+	 * @param classLoader The class loader used to resolve serialized classes from legacy savepoint formats.
+	 * @return The loaded savepoint
+	 *
+	 * @throws IOException Failures during load are forwarded
+	 */
+	public static Tuple2<Savepoint, StreamStateHandle> loadSavepointWithHandle(
+			String savepointFileOrDirectory,
+			ClassLoader classLoader) throws IOException {
+		
+		checkNotNull(savepointFileOrDirectory, "savepointFileOrDirectory");
+		checkNotNull(classLoader, "classLoader");
 
 		Path path = new Path(savepointFileOrDirectory);
 
@@ -180,11 +225,13 @@ public class SavepointStore {
 				LOG.info("Using savepoint file in {}", path);
 			} else {
 				throw new IOException("Cannot find meta data file in directory " + path
-					+ ". Please try to load the savepoint directly from the meta data file "
-					+ "instead of the directory.");
+						+ ". Please try to load the savepoint directly from the meta data file "
+						+ "instead of the directory.");
 			}
 		}
 
+		// load the savepoint
+		final Savepoint savepoint;
 		try (DataInputStream dis = new DataInputViewStreamWrapper(fs.open(path))) {
 			int magicNumber = dis.readInt();
 
@@ -192,15 +239,27 @@ public class SavepointStore {
 				int version = dis.readInt();
 
 				SavepointSerializer<?> serializer = SavepointSerializers.getSerializer(version);
-				return serializer.deserialize(dis, userClassLoader);
+				savepoint = serializer.deserialize(dis, classLoader);
 			} else {
-				throw new RuntimeException("Unexpected magic number. This is most likely " +
-						"caused by trying to load a Flink 1.0 savepoint. You cannot load a " +
-						"savepoint triggered by Flink 1.0 with this version of Flink. If it is " +
-						"_not_ a Flink 1.0 savepoint, this error indicates that the specified " +
-						"file is not a proper savepoint or the file has been corrupted.");
+				throw new RuntimeException("Unexpected magic number. This can have multiple reasons: " +
+						"(1) You are trying to load a Flink 1.0 savepoint, which is not supported by this " +
+						"version of Flink. (2) The file you were pointing to is not a savepoint at all. " +
+						"(3) The savepoint file has been corrupted.");
 			}
 		}
+
+		// construct the stream handle to the metadata file
+		// we get the size best-effort
+		long size = 0;
+		try {
+			size = fs.getFileStatus(path).getLen();
+		}
+		catch (Exception ignored) {
+			// we don't know the size, but we don't want to fail the savepoint loading for that
+		}
+		StreamStateHandle metadataHandle = new FileStateHandle(path, size);
+
+		return new Tuple2<>(savepoint, metadataHandle);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
index c6f5c86..b250831 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FutureUtil;
 
 import java.util.concurrent.RunnableFuture;
@@ -42,26 +43,22 @@ public class StateUtil {
 			Iterable<? extends StateObject> handlesToDiscard) throws Exception {
 
 		if (handlesToDiscard != null) {
-
-			Exception suppressedExceptions = null;
+			Exception exception = null;
 
 			for (StateObject state : handlesToDiscard) {
 
 				if (state != null) {
 					try {
 						state.discardState();
-					} catch (Exception ex) {
-						//best effort to still cleanup other states and deliver exceptions in the end
-						if (suppressedExceptions == null) {
-							suppressedExceptions = new Exception(ex);
-						}
-						suppressedExceptions.addSuppressed(ex);
+					}
+					catch (Exception ex) {
+						exception = ExceptionUtils.firstOrSuppressed(ex, exception);
 					}
 				}
 			}
 
-			if (suppressedExceptions != null) {
-				throw suppressedExceptions;
+			if (exception != null) {
+				throw exception;
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 21749cb..87cd4ac 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.jobmanager
 
-import java.io.{File, IOException}
+import java.io.IOException
 import java.net._
 import java.util.UUID
 import java.util.concurrent.{TimeUnit, Future => _, TimeoutException => _, _}
@@ -50,7 +50,7 @@ import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
 import org.apache.flink.runtime.executiongraph._
 import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceID, InstanceManager}
-import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus}
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway
@@ -77,7 +77,7 @@ import org.apache.flink.runtime.security.SecurityUtils.SecurityConfiguration
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils}
-import org.apache.flink.runtime.{FlinkActor, JobException, LeaderSessionMessageFilter, LogMessages}
+import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages}
 import org.apache.flink.util.{ConfigurationUtil, InstantiationUtil, NetUtils}
 import org.jboss.netty.channel.ChannelException
 
@@ -611,7 +611,7 @@ class JobManager(
                 new BiFunction[CompletedCheckpoint, Throwable, Void] {
                   override def apply(success: CompletedCheckpoint, cause: Throwable): Void = {
                     if (success != null) {
-                      val path = success.getExternalPath()
+                      val path = success.getExternalPointer()
                       log.info(s"Savepoint stored in $path. Now cancelling $jobId.")
                       executionGraph.cancel()
                       senderRef ! decorateMessage(CancellationSuccess(jobId, path))
@@ -787,11 +787,11 @@ class JobManager(
                 new BiFunction[CompletedCheckpoint, Throwable, Void] {
                   override def apply(success: CompletedCheckpoint, cause: Throwable): Void = {
                     if (success != null) {
-                      if (success.getExternalPath != null) {
+                      if (success.getExternalPointer != null) {
                         senderRef ! TriggerSavepointSuccess(
                           jobId,
                           success.getCheckpointID,
-                          success.getExternalPath,
+                          success.getExternalPointer,
                           success.getTimestamp
                         )
                       } else {
@@ -1784,7 +1784,7 @@ class JobManager(
       case t: Throwable =>
         log.error(s"Could not properly unregister job $jobID form the library cache.", t)
     }
-    jobManagerMetricGroup.map(_.removeJob(jobID))
+    jobManagerMetricGroup.foreach(_.removeJob(jobID))
 
     futureOption
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
index d4c3a2d..9517257 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
@@ -134,5 +134,10 @@ public class CheckpointCoordinatorFailureTest extends TestLogger {
 		public int getNumberOfRetainedCheckpoints() {
 			return -1;
 		}
+
+		@Override
+		public boolean requiresExternalizedCheckpoints() {
+			return false;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
index 725b85f..f77c755 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
@@ -237,7 +237,7 @@ public abstract class CompletedCheckpointStoreTest extends TestLogger {
 			Map<JobVertexID, TaskState> taskGroupStates,
 			CheckpointProperties props) {
 
-			super(jobId, checkpointId, timestamp, Long.MAX_VALUE, taskGroupStates, props, null);
+			super(jobId, checkpointId, timestamp, Long.MAX_VALUE, taskGroupStates, props);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java
index 0d933ff..b34e9a6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java
@@ -19,9 +19,11 @@
 package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -55,7 +57,9 @@ public class CompletedCheckpointTest {
 
 		// Verify discard call is forwarded to state
 		CompletedCheckpoint checkpoint = new CompletedCheckpoint(
-				new JobID(), 0, 0, 1, taskStates, CheckpointProperties.forStandardCheckpoint(), file.getAbsolutePath());
+				new JobID(), 0, 0, 1, taskStates, CheckpointProperties.forStandardCheckpoint(),
+				new FileStateHandle(new Path(file.toURI()), file.length()),
+				file.getAbsolutePath());
 
 		checkpoint.discard(JobStatus.FAILED);
 
@@ -74,7 +78,7 @@ public class CompletedCheckpointTest {
 		boolean discardSubsumed = true;
 		CheckpointProperties props = new CheckpointProperties(false, false, discardSubsumed, true, true, true, true);
 		CompletedCheckpoint checkpoint = new CompletedCheckpoint(
-				new JobID(), 0, 0, 1, taskStates, props, null);
+				new JobID(), 0, 0, 1, taskStates, props);
 
 		// Subsume
 		checkpoint.subsume();
@@ -104,7 +108,9 @@ public class CompletedCheckpointTest {
 			// Keep
 			CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false);
 			CompletedCheckpoint checkpoint = new CompletedCheckpoint(
-					new JobID(), 0, 0, 1, new HashMap<>(taskStates), props, externalPath);
+					new JobID(), 0, 0, 1, new HashMap<>(taskStates), props,
+					new FileStateHandle(new Path(file.toURI()), file.length()),
+					externalPath);
 
 			checkpoint.discard(status);
 			verify(state, times(0)).discardState();
@@ -113,7 +119,7 @@ public class CompletedCheckpointTest {
 			// Discard
 			props = new CheckpointProperties(false, false, true, true, true, true, true);
 			checkpoint = new CompletedCheckpoint(
-					new JobID(), 0, 0, 1, new HashMap<>(taskStates), props, null);
+					new JobID(), 0, 0, 1, new HashMap<>(taskStates), props);
 
 			checkpoint.discard(status);
 			verify(state, times(1)).discardState();
@@ -135,8 +141,7 @@ public class CompletedCheckpointTest {
 			0,
 			1,
 			new HashMap<>(taskStates),
-			CheckpointProperties.forStandardCheckpoint(),
-			null);
+			CheckpointProperties.forStandardCheckpoint());
 
 		CompletedCheckpointStats.DiscardCallback callback = mock(CompletedCheckpointStats.DiscardCallback.class);
 		completed.setDiscardCallback(callback);

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
index 3a85c4c..6f04f39 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
@@ -24,9 +24,11 @@ import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+
 import org.mockito.Mockito;
 
 import java.io.File;
@@ -49,9 +51,6 @@ import static org.mockito.Mockito.verify;
 
 public class PendingCheckpointTest {
 
-	@Rule
-	public TemporaryFolder tmpFolder = new TemporaryFolder();
-
 	private static final Map<ExecutionAttemptID, ExecutionVertex> ACK_TASKS = new HashMap<>();
 	private static final ExecutionAttemptID ATTEMPT_ID = new ExecutionAttemptID();
 
@@ -59,6 +58,9 @@ public class PendingCheckpointTest {
 		ACK_TASKS.put(ATTEMPT_ID, mock(ExecutionVertex.class));
 	}
 
+	@Rule
+	public final TemporaryFolder tmpFolder = new TemporaryFolder();
+
 	/**
 	 * Tests that pending checkpoints can be subsumed iff they are forced.
 	 */
@@ -96,7 +98,7 @@ public class PendingCheckpointTest {
 		PendingCheckpoint pending = createPendingCheckpoint(persisted, tmp.getAbsolutePath());
 		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
 		assertEquals(0, tmp.listFiles().length);
-		pending.finalizeCheckpoint();
+		pending.finalizeCheckpointExternalized();
 		assertEquals(1, tmp.listFiles().length);
 
 		// Ephemeral checkpoint
@@ -105,7 +107,7 @@ public class PendingCheckpointTest {
 		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
 
 		assertEquals(1, tmp.listFiles().length);
-		pending.finalizeCheckpoint();
+		pending.finalizeCheckpointNonExternalized();
 		assertEquals(1, tmp.listFiles().length);
 	}
 
@@ -148,7 +150,8 @@ public class PendingCheckpointTest {
 
 		assertFalse(future.isDone());
 		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
-		pending.finalizeCheckpoint();
+		assertTrue(pending.isFullyAcknowledged());
+		pending.finalizeCheckpointExternalized();
 		assertTrue(future.isDone());
 
 		// Finalize (missing ACKs)
@@ -157,7 +160,13 @@ public class PendingCheckpointTest {
 
 		assertFalse(future.isDone());
 		try {
-			pending.finalizeCheckpoint();
+			pending.finalizeCheckpointNonExternalized();
+			fail("Did not throw expected Exception");
+		} catch (IllegalStateException ignored) {
+			// Expected
+		}
+		try {
+			pending.finalizeCheckpointExternalized();
 			fail("Did not throw expected Exception");
 		} catch (IllegalStateException ignored) {
 			// Expected
@@ -233,7 +242,7 @@ public class PendingCheckpointTest {
 			pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
 			verify(callback, times(1)).reportSubtaskStats(any(JobVertexID.class), any(SubtaskStateStats.class));
 
-			pending.finalizeCheckpoint();
+			pending.finalizeCheckpointNonExternalized();
 			verify(callback, times(1)).reportCompletedCheckpoint(any(String.class));
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/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 5a38be2..cbb077c 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
@@ -483,6 +483,10 @@ public class JobManagerHARecoveryTest {
 			return checkpoints.size();
 		}
 
+		@Override
+		public boolean requiresExternalizedCheckpoints() {
+			return false;
+		}
 	}
 
 	static class MyCheckpointRecoveryFactory implements CheckpointRecoveryFactory {

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
index 60b12d2..75f1fd4 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
@@ -979,7 +979,7 @@ class JobManagerITCase(_system: ActorSystem)
           jobManager.tell(TriggerSavepoint(jobGraph.getJobID(), Option.apply("any")), testActor)
 
           val checkpoint = Mockito.mock(classOf[CompletedCheckpoint])
-          when(checkpoint.getExternalPath).thenReturn("Expected test savepoint path")
+          when(checkpoint.getExternalPointer).thenReturn("Expected test savepoint path")
 
           // Succeed the promise
           savepointPromise.complete(checkpoint)

http://git-wip-us.apache.org/repos/asf/flink/blob/5b7f21d8/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
index 60a3a62..f910e49 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
@@ -167,7 +167,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Parallelism);
 
 		ActorSystem testSystem = null;
-		JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2];
+		final JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2];
 		LeaderRetrievalService leaderRetrievalService = null;
 		ActorSystem taskManagerSystem = null;
 


[08/11] flink git commit: [FLINK-5501] [runtime] Make config key for ZooKeeper path for running registry consistent with other keys

Posted by se...@apache.org.
[FLINK-5501] [runtime] Make config key for ZooKeeper path for running registry consistent with other keys


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

Branch: refs/heads/master
Commit: 7f244b8d4267b50c88aef69f6fd915595f23b368
Parents: e0dede9
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Feb 28 10:46:40 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/configuration/HighAvailabilityOptions.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7f244b8d/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
index b883bc3..2cfb25a 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
@@ -125,7 +125,7 @@ public class HighAvailabilityOptions {
 			.withDeprecatedKeys("recovery.zookeeper.client.max-retry-attempts");
 
 	public static final ConfigOption<String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH = 
-			key("high-availability.zookeeper.job.registry")
+			key("high-availability.zookeeper.path.running-registry")
 			.defaultValue("/running_job_registry/");
 
 	// ------------------------------------------------------------------------


[10/11] flink git commit: [FLINK-5822] [state backends] Make JobManager / Checkpoint Coordinator aware of the root state backend

Posted by se...@apache.org.
[FLINK-5822] [state backends] Make JobManager / Checkpoint Coordinator aware of the root state backend

This closes #3411


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

Branch: refs/heads/master
Commit: 3446e66aac63a3dfdaf8cfd4a73bd80a7f038379
Parents: 5b7f21d
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Feb 17 17:51:00 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 19:02:13 2017 +0100

----------------------------------------------------------------------
 .../streaming/state/RocksDBStateBackend.java    |  12 +-
 .../state/RocksDBStateBackendFactory.java       |  19 +-
 .../jobmanager/JMXJobManagerMetricTest.java     |   2 +-
 .../CheckpointConfigHandlerTest.java            |   3 +
 .../checkpoint/CheckpointCoordinator.java       |  21 ++-
 .../runtime/executiongraph/ExecutionGraph.java  |   4 +-
 .../executiongraph/ExecutionGraphBuilder.java   |  32 +++-
 .../jobgraph/tasks/JobSnapshottingSettings.java |  15 +-
 .../runtime/state/AbstractStateBackend.java     | 173 ++++++++++++++++++-
 .../runtime/state/StateBackendFactory.java      |  16 +-
 .../state/filesystem/FsStateBackend.java        |  31 +++-
 .../state/filesystem/FsStateBackendFactory.java |  22 +--
 .../flink/runtime/state/heap/package-info.java  |  23 +++
 .../runtime/state/internal/package-info.java    |  52 ++++++
 .../state/memory/MemoryStateBackend.java        |   2 +-
 .../checkpoint/CheckpointStatsTrackerTest.java  |   1 +
 .../checkpoint/CoordinatorShutdownTest.java     |   5 +-
 ...ExecutionGraphCheckpointCoordinatorTest.java |   3 +-
 .../ArchivedExecutionGraphTest.java             |   3 +-
 .../tasks/JobSnapshottingSettingsTest.java      |   6 +
 .../jobmanager/JobManagerHARecoveryTest.java    |   1 +
 .../runtime/jobmanager/JobManagerTest.java      |   5 +
 .../flink/runtime/jobmanager/JobSubmitTest.java |   2 +-
 .../runtime/state/StateBackendLoadingTest.java  | 164 ++++++++++++++++++
 .../runtime/jobmanager/JobManagerITCase.scala   |   3 +
 .../api/graph/StreamGraphGenerator.java         |   2 +-
 .../api/graph/StreamingJobGraphGenerator.java   |   1 +
 .../streaming/runtime/tasks/StreamTask.java     |  72 ++------
 .../runtime/tasks/BlockingCheckpointsTest.java  |   2 +-
 .../streaming/runtime/tasks/StreamTaskTest.java |  56 +++---
 .../streaming/runtime/StateBackendITCase.java   |   2 +-
 31 files changed, 609 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
index 3fd5d0f..dd0e2f7 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
@@ -29,10 +29,12 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.util.AbstractID;
+
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.NativeLibraryLoader;
 import org.rocksdb.RocksDB;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -160,7 +162,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 
 	private void lazyInitializeForJob(
 			Environment env,
-			String operatorIdentifier) throws Exception {
+			String operatorIdentifier) throws IOException {
 
 		if (isInitialized) {
 			return;
@@ -193,7 +195,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 			}
 
 			if (dirs.isEmpty()) {
-				throw new Exception("No local storage directories available. " + errorMessage);
+				throw new IOException("No local storage directories available. " + errorMessage);
 			} else {
 				initializedDbBasePaths = dirs.toArray(new File[dirs.size()]);
 			}
@@ -235,7 +237,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 			TypeSerializer<K> keySerializer,
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
-			TaskKvStateRegistry kvStateRegistry) throws Exception {
+			TaskKvStateRegistry kvStateRegistry) throws IOException {
 
 		// first, make sure that the RocksDB JNI library is loaded
 		// we do this explicitly here to have better error handling
@@ -437,7 +439,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	//  static library loading utilities
 	// ------------------------------------------------------------------------
 
-	private void ensureRocksDBIsLoaded(String tempDirectory) throws Exception {
+	private void ensureRocksDBIsLoaded(String tempDirectory) throws IOException {
 		synchronized (RocksDBStateBackend.class) {
 			if (!rocksDbInitialized) {
 
@@ -488,7 +490,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 					}
 				}
 
-				throw new Exception("Could not load the native RocksDB library", lastException);
+				throw new IOException("Could not load the native RocksDB library", lastException);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
index 5002272..bd9bcaa 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.contrib.streaming.state;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.StateBackendFactory;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.io.IOException;
 
 /**
  * A factory that creates an {@link org.apache.flink.contrib.streaming.state.RocksDBStateBackend}
  * from a configuration.
  */
-public class RocksDBStateBackendFactory implements StateBackendFactory<FsStateBackend> {
+public class RocksDBStateBackendFactory implements StateBackendFactory<RocksDBStateBackend> {
 
 	protected static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackendFactory.class);
 
@@ -44,9 +45,11 @@ public class RocksDBStateBackendFactory implements StateBackendFactory<FsStateBa
 	public static final String ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.rocksdb.checkpointdir";
 
 	@Override
-	public AbstractStateBackend createFromConfig(Configuration config) throws Exception {
-		String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
-		String rocksdbLocalPath = config.getString(ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+	public RocksDBStateBackend createFromConfig(Configuration config) 
+			throws IllegalConfigurationException, IOException {
+
+		final String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+		final String rocksdbLocalPath = config.getString(ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
 
 		if (checkpointDirURI == null) {
 			throw new IllegalConfigurationException(
@@ -67,8 +70,8 @@ public class RocksDBStateBackendFactory implements StateBackendFactory<FsStateBa
 			return backend;
 		}
 		catch (IllegalArgumentException e) {
-			throw new Exception("Cannot initialize RocksDB State Backend with URI '"
-									+ checkpointDirURI + '.', e);
+			throw new IllegalConfigurationException(
+					"Cannot initialize RocksDB State Backend with URI '" + checkpointDirURI + '.', e);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
index b3b7dfc..1fdac65 100644
--- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
+++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
@@ -74,7 +74,7 @@ public class JMXJobManagerMetricTest {
 				Collections.<JobVertexID>emptyList(),
 				Collections.<JobVertexID>emptyList(),
 				Collections.<JobVertexID>emptyList(),
-				500, 500, 50, 5, ExternalizedCheckpointSettings.none(), true));
+				500, 500, 50, 5, ExternalizedCheckpointSettings.none(), null, true));
 
 			flink.waitForActorsToBeAlive();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
index e517c3c..95ced0a 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
@@ -56,6 +56,7 @@ public class CheckpointConfigHandlerTest {
 			minPause,
 			maxConcurrent,
 			externalized,
+			null,
 			true);
 
 		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
@@ -92,6 +93,7 @@ public class CheckpointConfigHandlerTest {
 			1212L,
 			12,
 			ExternalizedCheckpointSettings.none(),
+			null,
 			false); // at least once
 
 		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
@@ -122,6 +124,7 @@ public class CheckpointConfigHandlerTest {
 			1212L,
 			12,
 			externalizedSettings,
+			null,
 			false); // at least once
 
 		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 6da6f7d..0592e3d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -85,6 +85,12 @@ public class CheckpointCoordinator {
 	/** The job whose checkpoint this coordinator coordinates */
 	private final JobID job;
 
+	/** Default checkpoint properties **/
+	private final CheckpointProperties checkpointProperties;
+
+	/** The executor used for asynchronous calls, like potentially blocking I/O */
+	private final Executor executor;
+	
 	/** Tasks who need to be sent a message when a checkpoint is started */
 	private final ExecutionVertex[] tasksToTrigger;
 
@@ -101,7 +107,9 @@ public class CheckpointCoordinator {
 	 * accessing this don't block the job manager actor and run asynchronously. */
 	private final CompletedCheckpointStore completedCheckpointStore;
 
-	/** Default directory for persistent checkpoints; <code>null</code> if none configured. */
+	/** Default directory for persistent checkpoints; <code>null</code> if none configured.
+	 * THIS WILL BE REPLACED BY PROPER STATE-BACKEND METADATA WRITING */
+	@Nullable
 	private final String checkpointDirectory;
 
 	/** A list of recent checkpoint IDs, to identify late messages (vs invalid ones) */
@@ -154,11 +162,6 @@ public class CheckpointCoordinator {
 	@Nullable
 	private CheckpointStatsTracker statsTracker;
 
-	/** Default checkpoint properties **/
-	private final CheckpointProperties checkpointProperties;
-
-	private final Executor executor;
-
 	// --------------------------------------------------------------------------------------------
 
 	public CheckpointCoordinator(
@@ -173,7 +176,7 @@ public class CheckpointCoordinator {
 			ExecutionVertex[] tasksToCommitTo,
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore completedCheckpointStore,
-			String checkpointDirectory,
+			@Nullable String checkpointDirectory,
 			Executor executor) {
 
 		// sanity checks
@@ -211,6 +214,8 @@ public class CheckpointCoordinator {
 		this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
 		this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
 		this.checkpointDirectory = checkpointDirectory;
+		this.executor = checkNotNull(executor);
+
 		this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS);
 
 		this.timer = new Timer("Checkpoint Timer", true);
@@ -229,8 +234,6 @@ public class CheckpointCoordinator {
 		} catch (Throwable t) {
 			throw new RuntimeException("Failed to start checkpoint ID counter: " + t.getMessage(), t);
 		}
-
-		this.executor = checkNotNull(executor);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/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 ad4347d..a76a421 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
@@ -61,6 +61,7 @@ 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.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.runtime.util.SerializedThrowable;
@@ -348,7 +349,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		return false;
 	}
 
-	public void enableSnapshotCheckpointing(
+	public void enableCheckpointing(
 			long interval,
 			long checkpointTimeout,
 			long minPauseBetweenCheckpoints,
@@ -360,6 +361,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore checkpointStore,
 			String checkpointDir,
+			StateBackend metadataStore,
 			CheckpointStatsTracker statsTracker) {
 
 		// simple sanity checks

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
index c558e43..2a79302 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
@@ -37,6 +38,9 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.util.DynamicCodeLoadingException;
 import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
@@ -71,8 +75,8 @@ public class ExecutionGraphBuilder {
 			MetricGroup metrics,
 			int parallelismForAutoMax,
 			Logger log)
-		throws JobExecutionException, JobException
-	{
+		throws JobExecutionException, JobException {
+
 		checkNotNull(jobGraph, "job graph cannot be null");
 
 		final String jobName = jobGraph.getName();
@@ -191,7 +195,28 @@ public class ExecutionGraphBuilder {
 			String externalizedCheckpointsDir = jobManagerConfig.getString(
 					ConfigConstants.CHECKPOINTS_DIRECTORY_KEY, null);
 
-			executionGraph.enableSnapshotCheckpointing(
+			// load the state backend for checkpoint metadata.
+			// if specified in the application, use from there, otherwise load from configuration
+			final StateBackend metadataBackend;
+
+			final StateBackend applicationConfiguredBackend = snapshotSettings.getDefaultStateBackend();
+			if (applicationConfiguredBackend != null) {
+				metadataBackend = applicationConfiguredBackend;
+
+				log.info("Using application-defined state backend for checkpoint/savepoint metadata: {}.",
+						applicationConfiguredBackend);
+			}
+			else {
+				try {
+					metadataBackend = AbstractStateBackend
+							.loadStateBackendFromConfigOrCreateDefault(jobManagerConfig, classLoader, log);
+				}
+				catch (IllegalConfigurationException | IOException | DynamicCodeLoadingException e) {
+					throw new JobExecutionException(jobId, "Could not instantiate configured state backend", e);
+				}
+			}
+
+			executionGraph.enableCheckpointing(
 					snapshotSettings.getCheckpointInterval(),
 					snapshotSettings.getCheckpointTimeout(),
 					snapshotSettings.getMinPauseBetweenCheckpoints(),
@@ -203,6 +228,7 @@ public class ExecutionGraphBuilder {
 					checkpointIdCounter,
 					completedCheckpoints,
 					externalizedCheckpointsDir,
+					metadataBackend,
 					checkpointStatsTracker);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettings.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettings.java
index 561ba89..233aa88 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettings.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettings.java
@@ -19,7 +19,9 @@
 package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.StateBackend;
 
+import javax.annotation.Nullable;
 import java.util.List;
 
 import static java.util.Objects.requireNonNull;
@@ -50,6 +52,10 @@ public class JobSnapshottingSettings implements java.io.Serializable {
 	/** Settings for externalized checkpoints. */
 	private final ExternalizedCheckpointSettings externalizedCheckpointSettings;
 
+	/** The default state backend, if configured by the user in the job */
+	@Nullable
+	private final StateBackend defaultStateBackend;
+
 	/**
 	 * Flag indicating whether exactly once checkpoint mode has been configured.
 	 * If <code>false</code>, at least once mode has been configured. This is
@@ -58,7 +64,7 @@ public class JobSnapshottingSettings implements java.io.Serializable {
 	 * UI.
 	 */
 	private final boolean isExactlyOnce;
-	
+
 	public JobSnapshottingSettings(
 			List<JobVertexID> verticesToTrigger,
 			List<JobVertexID> verticesToAcknowledge,
@@ -68,6 +74,7 @@ public class JobSnapshottingSettings implements java.io.Serializable {
 			long minPauseBetweenCheckpoints,
 			int maxConcurrentCheckpoints,
 			ExternalizedCheckpointSettings externalizedCheckpointSettings,
+			@Nullable StateBackend defaultStateBackend,
 			boolean isExactlyOnce) {
 
 		// sanity checks
@@ -84,6 +91,7 @@ public class JobSnapshottingSettings implements java.io.Serializable {
 		this.minPauseBetweenCheckpoints = minPauseBetweenCheckpoints;
 		this.maxConcurrentCheckpoints = maxConcurrentCheckpoints;
 		this.externalizedCheckpointSettings = requireNonNull(externalizedCheckpointSettings);
+		this.defaultStateBackend = defaultStateBackend;
 		this.isExactlyOnce = isExactlyOnce;
 	}
 	
@@ -121,6 +129,11 @@ public class JobSnapshottingSettings implements java.io.Serializable {
 		return externalizedCheckpointSettings;
 	}
 
+	@Nullable
+	public StateBackend getDefaultStateBackend() {
+		return defaultStateBackend;
+	}
+
 	public boolean isExactlyOnce() {
 		return isExactlyOnce;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
index a335e45..2cf20a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
@@ -21,20 +21,50 @@ package org.apache.flink.runtime.state;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.DynamicCodeLoadingException;
+
+import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * An abstract base implementation of the {@link StateBackend} interface.
+ * 
+ * <p>
  */
 @PublicEvolving
 public abstract class AbstractStateBackend implements StateBackend, java.io.Serializable {
 
 	private static final long serialVersionUID = 4620415814639230247L;
 
+	// ------------------------------------------------------------------------
+	//  Configuration shortcut names
+	// ------------------------------------------------------------------------
+
+	/** The shortcut configuration name for the MemoryState backend that checkpoints to the JobManager */
+	public static final String MEMORY_STATE_BACKEND_NAME = "jobmanager";
+
+	/** The shortcut configuration name for the FileSystem State backend */ 
+	public static final String FS_STATE_BACKEND_NAME = "filesystem";
+
+	/** The shortcut configuration name for the RocksDB State Backend */
+	public static final String ROCKSDB_STATE_BACKEND_NAME = "rocksdb";
+
+	// ------------------------------------------------------------------------
+	//  State Backend - Persisting Byte Storage
+	// ------------------------------------------------------------------------
+
 	@Override
 	public abstract CheckpointStreamFactory createStreamFactory(
 			JobID jobId,
@@ -46,6 +76,10 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri
 			String operatorIdentifier,
 			@Nullable String targetLocation) throws IOException;
 
+	// ------------------------------------------------------------------------
+	//  State Backend - State-Holding Backends
+	// ------------------------------------------------------------------------
+
 	@Override
 	public abstract <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 			Environment env,
@@ -54,7 +88,7 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri
 			TypeSerializer<K> keySerializer,
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
-			TaskKvStateRegistry kvStateRegistry) throws Exception;
+			TaskKvStateRegistry kvStateRegistry) throws IOException;
 
 	@Override
 	public OperatorStateBackend createOperatorStateBackend(
@@ -63,4 +97,141 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri
 
 		return new DefaultOperatorStateBackend(env.getUserClassLoader());
 	}
+
+	// ------------------------------------------------------------------------
+	//  Loading the state backend from a configuration 
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Loads the state backend from the configuration, from the parameter 'state.backend', as defined
+	 * in {@link CoreOptions#STATE_BACKEND}.
+	 * 
+	 * <p>The state backends can be specified either via their shortcut name, or via the class name
+	 * of a {@link StateBackendFactory}. If a StateBackendFactory class name is specified, the factory
+	 * is instantiated (via its zero-argument constructor) and its
+	 * {@link StateBackendFactory#createFromConfig(Configuration)} method is called.
+	 *
+	 * <p>Recognized shortcut names are '{@value AbstractStateBackend#MEMORY_STATE_BACKEND_NAME}',
+	 * '{@value AbstractStateBackend#FS_STATE_BACKEND_NAME}', and
+	 * '{@value AbstractStateBackend#ROCKSDB_STATE_BACKEND_NAME}'.
+	 * 
+	 * @param config The configuration to load the state backend from
+	 * @param classLoader The class loader that should be used to load the state backend
+	 * @param logger Optionally, a logger to log actions to (may be null)
+	 * 
+	 * @return The instantiated state backend.
+	 * 
+	 * @throws DynamicCodeLoadingException
+	 *             Thrown if a state backend factory is configured and the factory class was not
+	 *             found or the factory could not be instantiated
+	 * @throws IllegalConfigurationException
+	 *             May be thrown by the StateBackendFactory when creating / configuring the state
+	 *             backend in the factory
+	 * @throws IOException
+	 *             May be thrown by the StateBackendFactory when instantiating the state backend
+	 */
+	public static StateBackend loadStateBackendFromConfig(
+			Configuration config,
+			ClassLoader classLoader,
+			@Nullable Logger logger) throws IllegalConfigurationException, DynamicCodeLoadingException, IOException {
+
+		checkNotNull(config, "config");
+		checkNotNull(classLoader, "classLoader");
+
+		final String backendName = config.getString(CoreOptions.STATE_BACKEND);
+		if (backendName == null) {
+			return null;
+		}
+
+		// by default the factory class is the backend name 
+		String factoryClassName = backendName;
+
+		switch (backendName.toLowerCase()) {
+			case MEMORY_STATE_BACKEND_NAME:
+				if (logger != null) {
+					logger.info("State backend is set to heap memory (checkpoint to JobManager)");
+				}
+				return new MemoryStateBackend();
+
+			case FS_STATE_BACKEND_NAME:
+				FsStateBackend fsBackend = new FsStateBackendFactory().createFromConfig(config);
+				if (logger != null) {
+					logger.info("State backend is set to heap memory (checkpoints to filesystem \"{}\")",
+							fsBackend.getBasePath());
+				}
+				return fsBackend;
+
+			case ROCKSDB_STATE_BACKEND_NAME:
+				factoryClassName = "org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory";
+				// fall through to the 'default' case that uses reflection to load the backend
+				// that way we can keep RocksDB in a separate module
+
+			default:
+				if (logger != null) {
+					logger.info("Loading state backend via factory {}", factoryClassName);
+				}
+
+				StateBackendFactory<?> factory;
+				try {
+					@SuppressWarnings("rawtypes")
+					Class<? extends StateBackendFactory> clazz = 
+							Class.forName(factoryClassName, false, classLoader)
+									.asSubclass(StateBackendFactory.class);
+
+					factory = clazz.newInstance();
+				}
+				catch (ClassNotFoundException e) {
+					throw new DynamicCodeLoadingException(
+							"Cannot find configured state backend factory class: " + backendName, e);
+				}
+				catch (ClassCastException | InstantiationException | IllegalAccessException e) {
+					throw new DynamicCodeLoadingException("The class configured under '" +
+							CoreOptions.STATE_BACKEND.key() + "' is not a valid state backend factory (" +
+							backendName + ')', e);
+				}
+				
+				return factory.createFromConfig(config);
+		}
+	}
+
+	/**
+	 * Loads the state backend from the configuration, from the parameter 'state.backend', as defined
+	 * in {@link CoreOptions#STATE_BACKEND}. If no state backend is configures, this instantiates the
+	 * default state backend (the {@link MemoryStateBackend}). 
+	 *
+	 * <p>Refer to {@link #loadStateBackendFromConfig(Configuration, ClassLoader, Logger)} for details on
+	 * how the state backend is loaded from the configuration.
+	 *
+	 * @param config The configuration to load the state backend from
+	 * @param classLoader The class loader that should be used to load the state backend
+	 * @param logger Optionally, a logger to log actions to (may be null)
+	 *
+	 * @return The instantiated state backend.
+	 *
+	 * @throws DynamicCodeLoadingException
+	 *             Thrown if a state backend factory is configured and the factory class was not
+	 *             found or the factory could not be instantiated
+	 * @throws IllegalConfigurationException
+	 *             May be thrown by the StateBackendFactory when creating / configuring the state
+	 *             backend in the factory
+	 * @throws IOException
+	 *             May be thrown by the StateBackendFactory when instantiating the state backend
+	 */
+	public static StateBackend loadStateBackendFromConfigOrCreateDefault(
+			Configuration config,
+			ClassLoader classLoader,
+			@Nullable Logger logger) throws IllegalConfigurationException, DynamicCodeLoadingException, IOException {
+
+		final StateBackend fromConfig = loadStateBackendFromConfig(config, classLoader, logger);
+
+		if (fromConfig != null) {
+			return fromConfig;
+		}
+		else {
+			if (logger != null) {
+				logger.info("No state backend has been configured, using default state backend (Memory / JobManager)");
+			}
+			return new MemoryStateBackend();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
index 39e7ed2..78c976a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
@@ -18,17 +18,24 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
 
+import java.io.IOException;
 import java.io.Serializable;
 
 /**
  * A factory to create a specific state backend. The state backend creation gets a Configuration
  * object that can be used to read further config values.
  * 
+ * <p>The state backend factory is typically specified in the configuration to produce a
+ * configured state backend.
+ * 
  * @param <T> The type of the state backend created.
  */
-public interface StateBackendFactory<T extends AbstractStateBackend> extends Serializable {
+@PublicEvolving
+public interface StateBackendFactory<T extends StateBackend> extends Serializable {
 
 	/**
 	 * Creates the state backend, optionally using the given configuration.
@@ -36,7 +43,10 @@ public interface StateBackendFactory<T extends AbstractStateBackend> extends Ser
 	 * @param config The Flink configuration (loaded by the TaskManager).
 	 * @return The created state backend. 
 	 * 
-	 * @throws Exception Exceptions during instantiation can be forwarded.
+	 * @throws IllegalConfigurationException
+	 *             If the configuration misses critical values, or specifies invalid values
+	 * @throws IOException
+	 *             If the state backend initialization failed due to an I/O exception
 	 */
-	AbstractStateBackend createFromConfig(Configuration config) throws Exception;
+	T createFromConfig(Configuration config) throws IllegalConfigurationException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
index b614d98..5e8a15d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
@@ -36,6 +36,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+
 /**
  * The file state backend is a state backend that stores the state of streaming jobs in a file system.
  *
@@ -139,17 +141,14 @@ public class FsStateBackend extends AbstractStateBackend {
 	 *                             rather than in files
 	 * 
 	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+	 * @throws IllegalArgumentException Thrown, if the {@code fileStateSizeThreshold} is out of bounds.
 	 */
 	public FsStateBackend(URI checkpointDataUri, int fileStateSizeThreshold) throws IOException {
-		if (fileStateSizeThreshold < 0) {
-			throw new IllegalArgumentException("The threshold for file state size must be zero or larger.");
-		}
-		if (fileStateSizeThreshold > MAX_FILE_STATE_THRESHOLD) {
-			throw new IllegalArgumentException("The threshold for file state size cannot be larger than " +
-				MAX_FILE_STATE_THRESHOLD);
-		}
+		checkArgument(fileStateSizeThreshold >= 0, "The threshold for file state size must be zero or larger.");
+		checkArgument(fileStateSizeThreshold <= MAX_FILE_STATE_THRESHOLD, 
+				"The threshold for file state size cannot be larger than %s", MAX_FILE_STATE_THRESHOLD);
+
 		this.fileStateThreshold = fileStateSizeThreshold;
-		
 		this.basePath = validateAndNormalizeUri(checkpointDataUri);
 	}
 
@@ -163,6 +162,19 @@ public class FsStateBackend extends AbstractStateBackend {
 		return basePath;
 	}
 
+	/**
+	 * Gets the threshold below which state is stored as part of the metadata, rather than in files.
+	 * This threshold ensures that the backend does not create a large amount of very small files,
+	 * where potentially the file pointers are larger than the state itself.
+	 * 
+	 * <p>By default, this threshold is {@value #DEFAULT_FILE_STATE_THRESHOLD}.
+	 * 
+	 * @return The file size threshold, in bytes.
+	 */
+	public int getMinFileSizeThreshold() {
+		return fileStateThreshold;
+	}
+
 	// ------------------------------------------------------------------------
 	//  initialization and cleanup
 	// ------------------------------------------------------------------------
@@ -189,7 +201,8 @@ public class FsStateBackend extends AbstractStateBackend {
 			TypeSerializer<K> keySerializer,
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
-			TaskKvStateRegistry kvStateRegistry) throws Exception {
+			TaskKvStateRegistry kvStateRegistry) throws IOException {
+
 		return new HeapKeyedStateBackend<>(
 				kvStateRegistry,
 				keySerializer,

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
index 042700c..4c933ef 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
@@ -23,6 +23,8 @@ import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.StateBackendFactory;
 
+import java.io.IOException;
+
 /**
  * A factory that creates an {@link org.apache.flink.runtime.state.filesystem.FsStateBackend}
  * from a configuration.
@@ -35,28 +37,26 @@ public class FsStateBackendFactory implements StateBackendFactory<FsStateBackend
 	/** The key under which the config stores the threshold for state to be store in memory,
 	 * rather than in files */
 	public static final String MEMORY_THRESHOLD_CONF_KEY = "state.backend.fs.memory-threshold";
-	
-	
+
+
 	@Override
-	public FsStateBackend createFromConfig(Configuration config) throws Exception {
-		String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
-		int memoryThreshold = config.getInteger(
+	public FsStateBackend createFromConfig(Configuration config) throws IllegalConfigurationException {
+		final String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+		final int memoryThreshold = config.getInteger(
 			MEMORY_THRESHOLD_CONF_KEY, FsStateBackend.DEFAULT_FILE_STATE_THRESHOLD);
-		
+
 		if (checkpointDirURI == null) {
 			throw new IllegalConfigurationException(
 					"Cannot create the file system state backend: The configuration does not specify the " +
 							"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
 		}
-		
+
 		try {
 			Path path = new Path(checkpointDirURI);
 			return new FsStateBackend(path.toUri(), memoryThreshold);
 		}
-		catch (IllegalArgumentException e) {
-			throw new Exception("Cannot initialize File System State Backend with URI '"
-					+ checkpointDirURI + '.', e);
+		catch (IOException | IllegalArgumentException e) {
+			throw new IllegalConfigurationException("Invalid configuration for the state backend", e);
 		}
-		
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/package-info.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/package-info.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/package-info.java
new file mode 100644
index 0000000..2f34ed8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains the classes for key/value state backends that store the state
+ * on the JVM heap as objects.
+ */
+package org.apache.flink.runtime.state.heap;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/package-info.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/package-info.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/package-info.java
new file mode 100644
index 0000000..fcc4df9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/package-info.java
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package holds the classes of the <b>internal state type hierarchy</b>.
+ *
+ * <p>The internal state classes give access to the namespace getters and setters and access to
+ * additional functionality, like raw value access or state merging.
+ *
+ * <p>The public API state hierarchy is intended to be programmed against by Flink applications.
+ * The internal state hierarchy holds all the auxiliary methods that are used by the runtime and not
+ * intended to be used by user applications. These internal methods are considered of limited use to users and
+ * only confusing, and are usually not regarded as stable across releases.
+ *
+ * <p>Each specific type in the internal state hierarchy extends the type from the public
+ * state hierarchy. The following illustrates the relationship between the public- and the internal
+ * hierarchy at the example of a subset of the classes:
+ *
+ * <pre>
+ *             State
+ *               |
+ *               +-------------------InternalKvState
+ *               |                         |
+ *          MergingState                   |
+ *               |                         |
+ *               +-----------------InternalMergingState
+ *               |                         |
+ *      +--------+------+                  |
+ *      |               |                  |
+ * ReducingState    ListState        +-----+-----------------+
+ *      |               |            |                       |
+ *      |               +-----------   -----------------InternalListState
+ *      |                            |
+ *      +------------------InternalReducingState
+ * </pre>
+ */
+package org.apache.flink.runtime.state.internal;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
index 2cc1164..6e6b034 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
@@ -90,7 +90,7 @@ public class MemoryStateBackend extends AbstractStateBackend {
 			TypeSerializer<K> keySerializer,
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
-			TaskKvStateRegistry kvStateRegistry) throws IOException {
+			TaskKvStateRegistry kvStateRegistry) {
 
 		return new HeapKeyedStateBackend<>(
 				kvStateRegistry,

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
index 9a39182..7ab71cb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
@@ -62,6 +62,7 @@ public class CheckpointStatsTrackerTest {
 			191929L,
 			123,
 			ExternalizedCheckpointSettings.none(),
+			null,
 			false);
 
 		CheckpointStatsTracker tracker = new CheckpointStatsTracker(

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
index 7949ef0..976da48 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
@@ -32,7 +32,6 @@ import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testtasks.NoOpInvokable;
 
 import org.junit.Test;
 
@@ -67,7 +66,7 @@ public class CoordinatorShutdownTest {
 			
 			JobGraph testGraph = new JobGraph("test job", vertex);
 			testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 
-					5000, 60000, 0L, Integer.MAX_VALUE, ExternalizedCheckpointSettings.none(), true));
+					5000, 60000, 0L, Integer.MAX_VALUE, ExternalizedCheckpointSettings.none(), null, true));
 			
 			ActorGateway jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 
@@ -126,7 +125,7 @@ public class CoordinatorShutdownTest {
 
 			JobGraph testGraph = new JobGraph("test job", vertex);
 			testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList,
-					5000, 60000, 0L, Integer.MAX_VALUE, ExternalizedCheckpointSettings.none(), true));
+					5000, 60000, 0L, Integer.MAX_VALUE, ExternalizedCheckpointSettings.none(), null, true));
 			
 			ActorGateway jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
index 47e6826..8f565dd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
@@ -106,7 +106,7 @@ public class ExecutionGraphCheckpointCoordinatorTest {
 			ClassLoader.getSystemClassLoader(),
 			new UnregisteredMetricsGroup());
 
-		executionGraph.enableSnapshotCheckpointing(
+		executionGraph.enableCheckpointing(
 				100,
 				100,
 				100,
@@ -118,6 +118,7 @@ public class ExecutionGraphCheckpointCoordinatorTest {
 				counter,
 				store,
 				null,
+				null,
 				CheckpointStatsTrackerTest.createTestTracker());
 
 		JobVertex jobVertex = new JobVertex("MockVertex");

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
index 46ce3f4..3090172 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
@@ -112,7 +112,7 @@ public class ArchivedExecutionGraphTest {
 				mock(JobSnapshottingSettings.class),
 				new UnregisteredMetricsGroup());
 
-		runtimeGraph.enableSnapshotCheckpointing(
+		runtimeGraph.enableCheckpointing(
 			100,
 			100,
 			100,
@@ -124,6 +124,7 @@ public class ArchivedExecutionGraphTest {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			null,
 			statsTracker);
 
 		Map<String, Accumulator<?, ?>> userAccumulators = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettingsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettingsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettingsTest.java
index 667dbca..2508d5c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettingsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobSnapshottingSettingsTest.java
@@ -20,11 +20,14 @@ package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.junit.Test;
 
 import java.util.Arrays;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class JobSnapshottingSettingsTest {
 
@@ -42,6 +45,7 @@ public class JobSnapshottingSettingsTest {
 			112,
 			12,
 			ExternalizedCheckpointSettings.externalizeCheckpoints(true),
+			new MemoryStateBackend(),
 			false);
 
 		JobSnapshottingSettings copy = CommonTestUtils.createCopySerializable(settings);
@@ -55,5 +59,7 @@ public class JobSnapshottingSettingsTest {
 		assertEquals(settings.getExternalizedCheckpointSettings().externalizeCheckpoints(), copy.getExternalizedCheckpointSettings().externalizeCheckpoints());
 		assertEquals(settings.getExternalizedCheckpointSettings().deleteOnCancellation(), copy.getExternalizedCheckpointSettings().deleteOnCancellation());
 		assertEquals(settings.isExactlyOnce(), copy.isExactlyOnce());
+		assertNotNull(copy.getDefaultStateBackend());
+		assertTrue(copy.getDefaultStateBackend().getClass() == MemoryStateBackend.class);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/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 cbb077c..115b06c 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
@@ -225,6 +225,7 @@ public class JobManagerHARecoveryTest {
 					0,
 					1,
 					ExternalizedCheckpointSettings.none(),
+					null,
 					true));
 
 			BlockingStatefulInvokable.initializeStaticHelpers(slots);

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
index c5f6d99..727fc65 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
@@ -829,6 +829,7 @@ public class JobManagerTest {
 					0,
 					Integer.MAX_VALUE,
 					ExternalizedCheckpointSettings.none(),
+					null,
 					true);
 
 			jobGraph.setSnapshotSettings(snapshottingSettings);
@@ -954,6 +955,7 @@ public class JobManagerTest {
 				0,
 				Integer.MAX_VALUE,
 				ExternalizedCheckpointSettings.none(),
+				null,
 				true);
 
 			jobGraph.setSnapshotSettings(snapshottingSettings);
@@ -1059,6 +1061,7 @@ public class JobManagerTest {
 					0,
 					Integer.MAX_VALUE,
 					ExternalizedCheckpointSettings.none(),
+					null,
 					true);
 
 			jobGraph.setSnapshotSettings(snapshottingSettings);
@@ -1161,6 +1164,7 @@ public class JobManagerTest {
 					0,
 					Integer.MAX_VALUE,
 					ExternalizedCheckpointSettings.none(),
+					null,
 					true);
 
 			jobGraph.setSnapshotSettings(snapshottingSettings);
@@ -1207,6 +1211,7 @@ public class JobManagerTest {
 					0,
 					Integer.MAX_VALUE,
 					ExternalizedCheckpointSettings.none(),
+					null,
 					true);
 
 			newJobGraph.setSnapshotSettings(newSnapshottingSettings);

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
index feb3d4d..529c100 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
@@ -229,7 +229,7 @@ public class JobSubmitTest {
 
 		JobGraph jg = new JobGraph("test job", jobVertex);
 		jg.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList,
-			5000, 5000, 0L, 10, ExternalizedCheckpointSettings.none(), true));
+			5000, 5000, 0L, 10, ExternalizedCheckpointSettings.none(), null, true));
 		return jg;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java
new file mode 100644
index 0000000..a64faf1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.DynamicCodeLoadingException;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This test validates that state backends are properly loaded from configuration.
+ */
+public class StateBackendLoadingTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	private final ClassLoader cl = getClass().getClassLoader();
+
+	private final String backendKey = CoreOptions.STATE_BACKEND.key();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testNoStateBackendDefined() throws Exception {
+		assertNull(AbstractStateBackend.loadStateBackendFromConfig(new Configuration(), cl, null));
+	}
+
+	@Test
+	public void testInstantiateMemoryBackendByDefault() throws Exception {
+		StateBackend backend = AbstractStateBackend
+				.loadStateBackendFromConfigOrCreateDefault(new Configuration(), cl, null);
+
+		assertTrue(backend instanceof MemoryStateBackend);
+	}
+
+	@Test
+	public void testLoadMemoryStateBackend() throws Exception {
+		// we configure with the explicit string (rather than AbstractStateBackend#X_STATE_BACKEND_NAME)
+		// to guard against config-breaking changes of the name 
+		final Configuration config = new Configuration();
+		config.setString(backendKey, "jobmanager");
+
+		StateBackend backend = AbstractStateBackend
+				.loadStateBackendFromConfigOrCreateDefault(new Configuration(), cl, null);
+
+		assertTrue(backend instanceof MemoryStateBackend);
+	}
+
+	@Test
+	public void testLoadFileSystemStateBackend() throws Exception {
+		final String checkpointDir = new Path(tmp.getRoot().toURI()).toString();
+		final Path expectedPath = new Path(checkpointDir);
+		final int threshold = 1000000;
+
+		// we configure with the explicit string (rather than AbstractStateBackend#X_STATE_BACKEND_NAME)
+		// to guard against config-breaking changes of the name 
+		final Configuration config1 = new Configuration();
+		config1.setString(backendKey, "filesystem");
+		config1.setString("state.checkpoints.dir", checkpointDir);
+		config1.setString("state.backend.fs.checkpointdir", checkpointDir);
+		config1.setInteger("state.backend.fs.memory-threshold", threshold);
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, FsStateBackendFactory.class.getName());
+		config2.setString("state.checkpoints.dir", checkpointDir);
+		config2.setString("state.backend.fs.checkpointdir", checkpointDir);
+		config2.setInteger("state.backend.fs.memory-threshold", threshold);
+
+		StateBackend backend1 = AbstractStateBackend
+				.loadStateBackendFromConfigOrCreateDefault(config1, cl, null);
+
+		StateBackend backend2 = AbstractStateBackend
+				.loadStateBackendFromConfigOrCreateDefault(config2, cl, null);
+
+		assertTrue(backend1 instanceof FsStateBackend);
+		assertTrue(backend2 instanceof FsStateBackend);
+
+		FsStateBackend fs1 = (FsStateBackend) backend1;
+		FsStateBackend fs2 = (FsStateBackend) backend2;
+
+		assertEquals(expectedPath, fs1.getBasePath());
+		assertEquals(expectedPath, fs2.getBasePath());
+		assertEquals(threshold, fs1.getMinFileSizeThreshold());
+		assertEquals(threshold, fs2.getMinFileSizeThreshold());
+	}
+
+	/**
+	 * This test makes sure that failures properly manifest when the state backend could not be loaded.
+	 */
+	@Test
+	public void testLoadingFails() throws Exception {
+		final Configuration config = new Configuration();
+
+		// try a value that is neither recognized as a name, nor corresponds to a class
+		config.setString(backendKey, "does.not.exist");
+		try {
+			AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(config, cl, null);
+			fail("should fail with an exception");
+		} catch (DynamicCodeLoadingException ignored) {
+			// expected
+		}
+
+		// try a class that is not a factory
+		config.setString(backendKey, java.io.File.class.getName());
+		try {
+			AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(config, cl, null);
+			fail("should fail with an exception");
+		} catch (DynamicCodeLoadingException ignored) {
+			// expected
+		}
+
+		// a factory that fails
+		config.setString(backendKey, FailingFactory.class.getName());
+		try {
+			AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(config, cl, null);
+			fail("should fail with an exception");
+		} catch (IOException ignored) {
+			// expected
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	static final class FailingFactory implements StateBackendFactory<StateBackend> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public StateBackend createFromConfig(Configuration config) throws IllegalConfigurationException, IOException {
+			throw new IOException("fail!");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
index 75f1fd4..31e72dd 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
@@ -822,6 +822,7 @@ class JobManagerITCase(_system: ActorSystem)
             60000,
             1,
             ExternalizedCheckpointSettings.none,
+            null,
             true))
 
           // Submit job...
@@ -881,6 +882,7 @@ class JobManagerITCase(_system: ActorSystem)
             60000,
             1,
             ExternalizedCheckpointSettings.none,
+            null,
             true))
 
           // Submit job...
@@ -948,6 +950,7 @@ class JobManagerITCase(_system: ActorSystem)
             60000,
             1,
             ExternalizedCheckpointSettings.none,
+            null,
             true))
 
           // Submit job...

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
index f55ff47..bd018c3 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -82,7 +82,7 @@ public class StreamGraphGenerator {
 	public static final int UPPER_BOUND_MAX_PARALLELISM = KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM;
 
 	// The StreamGraph that is being built, this is initialized at the beginning.
-	private StreamGraph streamGraph;
+	private final StreamGraph streamGraph;
 
 	private final StreamExecutionEnvironment env;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index a4bb165..003eff9 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -539,6 +539,7 @@ public class StreamingJobGraphGenerator {
 				cfg.getCheckpointTimeout(), cfg.getMinPauseBetweenCheckpoints(),
 				cfg.getMaxConcurrentCheckpoints(),
 				externalizedCheckpointSettings,
+				streamGraph.getStateBackend(),
 				isExactlyOnce);
 
 		jobGraph.setSnapshotSettings(settings);

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 938ffd2..1e208ee 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -20,9 +20,6 @@ package org.apache.flink.streaming.runtime.tasks;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
@@ -43,13 +40,10 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
-import org.apache.flink.runtime.state.StateBackendFactory;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.StateUtil;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateHandles;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -63,6 +57,7 @@ import org.apache.flink.util.CollectionUtil;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FutureUtil;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -147,7 +142,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 	private StreamConfig configuration;
 
 	/** Our state backend. We use this to create checkpoint streams and a keyed state backend. */
-	private AbstractStateBackend stateBackend;
+	private StateBackend stateBackend;
 
 	/** Keyed state backend for the head operator, if it is keyed. There can only ever be one. */
 	private AbstractKeyedStateBackend<?> keyedStateBackend;
@@ -713,61 +708,20 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 	//  State backend
 	// ------------------------------------------------------------------------
 
-	private AbstractStateBackend createStateBackend() throws Exception {
-		AbstractStateBackend stateBackend = configuration.getStateBackend(getUserCodeClassLoader());
+	private StateBackend createStateBackend() throws Exception {
+		final StateBackend fromJob = configuration.getStateBackend(getUserCodeClassLoader());
 
-		if (stateBackend != null) {
+		if (fromJob != null) {
 			// backend has been configured on the environment
 			LOG.info("Using user-defined state backend: {}.", stateBackend);
-		} else {
-			// see if we have a backend specified in the configuration
-			Configuration flinkConfig = getEnvironment().getTaskManagerInfo().getConfiguration();
-			String backendName = flinkConfig.getString(CoreOptions.STATE_BACKEND, null);
-
-			if (backendName == null) {
-				LOG.warn("No state backend has been specified, using default state backend (Memory / JobManager)");
-				backendName = "jobmanager";
-			}
-
-			switch (backendName.toLowerCase()) {
-				case "jobmanager":
-					LOG.info("State backend is set to heap memory (checkpoint to jobmanager)");
-					stateBackend = new MemoryStateBackend();
-					break;
-
-				case "filesystem":
-					FsStateBackend backend = new FsStateBackendFactory().createFromConfig(flinkConfig);
-					LOG.info("State backend is set to heap memory (checkpoints to filesystem \"{}\")",
-						backend.getBasePath());
-					stateBackend = backend;
-					break;
-
-				case "rocksdb":
-					backendName = "org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory";
-					// fall through to the 'default' case that uses reflection to load the backend
-					// that way we can keep RocksDB in a separate module
-
-				default:
-					try {
-						@SuppressWarnings("rawtypes")
-						Class<? extends StateBackendFactory> clazz =
-								Class.forName(backendName, false, getUserCodeClassLoader()).
-										asSubclass(StateBackendFactory.class);
-
-						stateBackend = clazz.newInstance().createFromConfig(flinkConfig);
-					} catch (ClassNotFoundException e) {
-						throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName);
-					} catch (ClassCastException e) {
-						throw new IllegalConfigurationException("The class configured under '" +
-								CoreOptions.STATE_BACKEND.key() + "' is not a valid state backend factory (" +
-								backendName + ')');
-					} catch (Throwable t) {
-						throw new IllegalConfigurationException("Cannot create configured state backend", t);
-					}
-			}
+			return fromJob;
+		}
+		else {
+			return AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(
+					getEnvironment().getTaskManagerInfo().getConfiguration(),
+					getUserCodeClassLoader(),
+					LOG);
 		}
-
-		return stateBackend;
 	}
 
 	public OperatorStateBackend createOperatorStateBackend(

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java
index 51294ce..e266ea1 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java
@@ -183,7 +183,7 @@ public class BlockingCheckpointsTest {
 		public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 				Environment env, JobID jobID, String operatorIdentifier,
 				TypeSerializer<K> keySerializer, int numberOfKeyGroups,
-				KeyGroupRange keyGroupRange, TaskKvStateRegistry kvStateRegistry) throws Exception {
+				KeyGroupRange keyGroupRange, TaskKvStateRegistry kvStateRegistry) {
 
 			throw new UnsupportedOperationException();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index 3d01fdd..3826051 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -807,33 +807,39 @@ public class StreamTaskTest extends TestLogger {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public AbstractStateBackend createFromConfig(Configuration config) throws Exception {
+		public AbstractStateBackend createFromConfig(Configuration config) {
 			AbstractStateBackend stateBackendMock = mock(AbstractStateBackend.class);
 
-			Mockito.when(stateBackendMock.createOperatorStateBackend(
-					Mockito.any(Environment.class),
-					Mockito.any(String.class)))
-				.thenAnswer(new Answer<OperatorStateBackend>() {
-					@Override
-					public OperatorStateBackend answer(InvocationOnMock invocationOnMock) throws Throwable {
-						return Mockito.mock(OperatorStateBackend.class);
-					}
-				});
-
-			Mockito.when(stateBackendMock.createKeyedStateBackend(
-					Mockito.any(Environment.class),
-					Mockito.any(JobID.class),
-					Mockito.any(String.class),
-					Mockito.any(TypeSerializer.class),
-					Mockito.any(int.class),
-					Mockito.any(KeyGroupRange.class),
-					Mockito.any(TaskKvStateRegistry.class)))
-				.thenAnswer(new Answer<AbstractKeyedStateBackend>() {
-					@Override
-					public AbstractKeyedStateBackend answer(InvocationOnMock invocationOnMock) throws Throwable {
-						return Mockito.mock(AbstractKeyedStateBackend.class);
-					}
-				});
+			try {
+				Mockito.when(stateBackendMock.createOperatorStateBackend(
+						Mockito.any(Environment.class),
+						Mockito.any(String.class)))
+					.thenAnswer(new Answer<OperatorStateBackend>() {
+						@Override
+						public OperatorStateBackend answer(InvocationOnMock invocationOnMock) throws Throwable {
+							return Mockito.mock(OperatorStateBackend.class);
+						}
+					});
+	
+				Mockito.when(stateBackendMock.createKeyedStateBackend(
+						Mockito.any(Environment.class),
+						Mockito.any(JobID.class),
+						Mockito.any(String.class),
+						Mockito.any(TypeSerializer.class),
+						Mockito.any(int.class),
+						Mockito.any(KeyGroupRange.class),
+						Mockito.any(TaskKvStateRegistry.class)))
+					.thenAnswer(new Answer<AbstractKeyedStateBackend>() {
+						@Override
+						public AbstractKeyedStateBackend answer(InvocationOnMock invocationOnMock) throws Throwable {
+							return Mockito.mock(AbstractKeyedStateBackend.class);
+						}
+					});
+			}
+			catch (Exception e) {
+				// this is needed, because the signatures of the mocked methods throw 'Exception'
+				throw new RuntimeException(e);
+			}
 
 			return stateBackendMock;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/3446e66a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
index 79665dd..4677242 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
@@ -109,7 +109,7 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase {
 				TypeSerializer<K> keySerializer,
 				int numberOfKeyGroups,
 				KeyGroupRange keyGroupRange,
-				TaskKvStateRegistry kvStateRegistry) throws Exception {
+				TaskKvStateRegistry kvStateRegistry) throws IOException {
 			throw new SuccessException();
 		}
 	}


[07/11] flink git commit: [FLINK-5133] [core] Followups for ResourceSpec on DataSet / DataStream API

Posted by se...@apache.org.
[FLINK-5133] [core] Followups for ResourceSpec on DataSet / DataStream API

  - Correct some use of Preconditions.checkNotNull
  - Make 'resources' plural in all cases
  - Add comments to why the setters are commented out
  - Add @PublicEvolving annotations
  - Make the Scala API completeness test match Scala-esk versions of Java Getters


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

Branch: refs/heads/master
Commit: 9912de21a1053013a220707f8b3868bdbf93aaca
Parents: f37ed02
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Feb 28 10:42:16 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../flink/api/common/operators/Operator.java    |  41 ++++---
 .../api/common/operators/ResourceSpec.java      |  29 ++---
 .../flink/api/java/operators/DataSink.java      | 109 ++++++++++---------
 .../api/java/operators/DeltaIteration.java      | 105 +++++++++---------
 .../flink/api/java/operators/Operator.java      | 101 +++++++++--------
 .../api/java/operators/OperatorTranslation.java |  20 ++--
 .../apache/flink/optimizer/plan/PlanNode.java   |   8 +-
 .../org/apache/flink/api/scala/DataSet.scala    |  69 ++++++------
 .../streaming/api/datastream/DataStream.java    |  18 +--
 .../api/datastream/DataStreamSink.java          |  76 +++++++------
 .../datastream/SingleOutputStreamOperator.java  |  74 +++++++------
 .../flink/streaming/api/graph/StreamGraph.java  |   2 +-
 .../api/graph/StreamGraphGenerator.java         |   4 +-
 .../flink/streaming/api/graph/StreamNode.java   |  18 +--
 .../transformations/StreamTransformation.java   |  30 ++---
 .../flink/streaming/api/scala/DataStream.scala  |  58 +++++-----
 .../ScalaAPICompletenessTestBase.scala          |  36 +++---
 17 files changed, 424 insertions(+), 374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
index a9dedfa..1905555 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
@@ -16,18 +16,20 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.operators;
 
 import java.util.List;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.operators.util.UserCodeWrapper;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Visitable;
 
+import javax.annotation.Nullable;
+
 /**
 * Abstract base class for all operators. An operator is a source, sink, or it applies an operation to
 * one or more inputs, producing a result.
@@ -45,9 +47,11 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 		
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;  // the number of parallel instances to use
 
-	private ResourceSpec minResource;			// the minimum resource of the contract instance.
+	@Nullable
+	private ResourceSpec minResources;          // the minimum resource of the contract instance.
 
-	private ResourceSpec preferredResource;	// the preferred resource of the contract instance.
+	@Nullable
+	private ResourceSpec preferredResources;    // the preferred resource of the contract instance.
 
 	/**
 	 * The return type of the user function.
@@ -190,35 +194,40 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	}
 
 	/**
-	 * Gets the minimum resource for this contract instance. The minimum resource denotes how many
-	 * resources will be needed in the minimum for the user function during the execution.
+	 * Gets the minimum resources for this operator. The minimum resources denotes how many
+	 * resources will be needed at least minimum for the operator or user function during the execution.
 	 *
-	 * @return The minimum resource of this operator.
+	 * @return The minimum resources of this operator.
 	 */
-	public ResourceSpec getMinResource() {
-		return this.minResource;
+	@Nullable
+	@PublicEvolving
+	public ResourceSpec getMinResources() {
+		return this.minResources;
 	}
 
 	/**
-	 * Gets the preferred resource for this contract instance. The preferred resource denotes how many
+	 * Gets the preferred resources for this contract instance. The preferred resources denote how many
 	 * resources will be needed in the maximum for the user function during the execution.
 	 *
 	 * @return The preferred resource of this operator.
 	 */
-	public ResourceSpec getPreferredResource() {
-		return this.preferredResource;
+	@Nullable
+	@PublicEvolving
+	public ResourceSpec getPreferredResources() {
+		return this.preferredResources;
 	}
 
 	/**
 	 * Sets the minimum and preferred resources for this contract instance. The resource denotes
 	 * how many memories and cpu cores of the user function will be consumed during the execution.
 	 *
-	 * @param minResource The minimum resource of this operator.
-	 * @param preferredResource The preferred resource of this operator.
+	 * @param minResources The minimum resource of this operator.
+	 * @param preferredResources The preferred resource of this operator.
 	 */
-	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
+	@PublicEvolving
+	public void setResource(ResourceSpec minResources, ResourceSpec preferredResources) {
+		this.minResources = minResources;
+		this.preferredResources = preferredResources;
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java
index 1387508..0ea289a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java
@@ -105,14 +105,12 @@ public class ResourceSpec implements Serializable {
 	 * @return The new resource with merged values.
 	 */
 	public ResourceSpec merge(ResourceSpec other) {
-		ResourceSpec result = new ResourceSpec(
+		return new ResourceSpec(
 				Math.max(this.cpuCores, other.cpuCores),
 				this.heapMemoryInMB + other.heapMemoryInMB,
 				this.directMemoryInMB + other.directMemoryInMB,
 				this.nativeMemoryInMB + other.nativeMemoryInMB,
 				this.stateSizeInMB + other.stateSizeInMB);
-
-		return  result;
 	}
 
 	public double getCpuCores() {
@@ -141,12 +139,8 @@ public class ResourceSpec implements Serializable {
 	 * @return True if all the values are equal or greater than 0, otherwise false.
 	 */
 	public boolean isValid() {
-		if (this.cpuCores >= 0 && this.heapMemoryInMB >= 0 && this.directMemoryInMB >= 0 &&
-				this.nativeMemoryInMB >= 0 && this.stateSizeInMB >= 0) {
-			return true;
-		} else {
-			return false;
-		}
+		return (this.cpuCores >= 0 && this.heapMemoryInMB >= 0 && this.directMemoryInMB >= 0 &&
+				this.nativeMemoryInMB >= 0 && this.stateSizeInMB >= 0);
 	}
 
 	/**
@@ -162,11 +156,7 @@ public class ResourceSpec implements Serializable {
 		int cmp3 = Integer.compare(this.directMemoryInMB, other.directMemoryInMB);
 		int cmp4 = Integer.compare(this.nativeMemoryInMB, other.nativeMemoryInMB);
 		int cmp5 = Integer.compare(this.stateSizeInMB, other.stateSizeInMB);
-		if (cmp1 <= 0 && cmp2 <= 0 && cmp3 <= 0 && cmp4 <= 0 && cmp5 <= 0) {
-			return true;
-		} else {
-			return false;
-		}
+		return (cmp1 <= 0 && cmp2 <= 0 && cmp3 <= 0 && cmp4 <= 0 && cmp5 <= 0);
 	}
 
 	@Override
@@ -186,6 +176,17 @@ public class ResourceSpec implements Serializable {
 	}
 
 	@Override
+	public int hashCode() {
+		final long cpuBits =  Double.doubleToLongBits(cpuCores);
+		int result = (int) (cpuBits ^ (cpuBits >>> 32));
+		result = 31 * result + heapMemoryInMB;
+		result = 31 * result + directMemoryInMB;
+		result = 31 * result + nativeMemoryInMB;
+		result = 31 * result + stateSizeInMB;
+		return result;
+	}
+
+	@Override
 	public String toString() {
 		return "ResourceSpec{" +
 				"cpuCores=" + cpuCores +

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
index 3be9cc0..369e013 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
@@ -52,9 +52,9 @@ public class DataSink<T> {
 	
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
 
-	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec minResources = ResourceSpec.UNKNOWN;
 
-	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec preferredResources = ResourceSpec.UNKNOWN;
 
 	private Configuration parameters;
 
@@ -285,61 +285,70 @@ public class DataSink<T> {
 	}
 
 	/**
-	 * Returns the minimum resource of this data sink. If no minimum resource has been set,
-	 * it returns the default empty resource.
+	 * Returns the minimum resources of this data sink. If no minimum resources have been set,
+	 * this returns the default resource profile.
 	 *
-	 * @return The minimum resource of this data sink.
+	 * @return The minimum resources of this data sink.
 	 */
-	public ResourceSpec getMinResource() {
-		return this.minResource;
+	@PublicEvolving
+	public ResourceSpec getMinResources() {
+		return this.minResources;
 	}
 
 	/**
-	 * Returns the preferred resource of this data sink. If no preferred resource has been set,
-	 * it returns the default empty resource.
+	 * Returns the preferred resources of this data sink. If no preferred resources have been set,
+	 * this returns the default resource profile.
 	 *
-	 * @return The preferred resource of this data sink.
+	 * @return The preferred resources of this data sink.
 	 */
-	public ResourceSpec getPreferredResource() {
-		return this.preferredResource;
+	@PublicEvolving
+	public ResourceSpec getPreferredResources() {
+		return this.preferredResources;
 	}
 
-	/**
-	 * Sets the minimum and preferred resources for this data sink. This overrides the default empty resource.
-	 *	The minimum resource must be satisfied and the preferred resource specifies the upper bound
-	 * for dynamic resource resize.
-	 *
-	 * @param minResource The minimum resource for this data sink.
-	 * @param preferredResource The preferred resource for this data sink.
-	 * @return The data sink with set minimum and preferred resources.
-	 */
-	/*
-	public DataSink<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		Preconditions.checkNotNull(minResource != null && preferredResource != null,
-				"The min and preferred resources must be not null.");
-		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
-				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
-
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
-
-		return this;
-	}*/
-
-	/**
-	 * Sets the resource for this data sink. This overrides the default empty minimum and preferred resources.
-	 *
-	 * @param resource The resource for this data sink.
-	 * @return The data sink with set minimum and preferred resources.
-	 */
-	/*
-	public DataSink<T> setResource(ResourceSpec resource) {
-		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
-		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
-
-		this.minResource = resource;
-		this.preferredResource = resource;
-
-		return this;
-	}*/
+//	---------------------------------------------------------------------------
+//	 Fine-grained resource profiles are an incomplete work-in-progress feature
+//	 The setters are hence commented out at this point.
+//	---------------------------------------------------------------------------
+//
+//	/**
+//	 * Sets the minimum and preferred resources for this data sink. This overrides the default empty resource.
+//	 *	The minimum resource must be satisfied and the preferred resource specifies the upper bound
+//	 * for dynamic resource resize.
+//	 *
+//	 * @param minResources The minimum resource for this data sink.
+//	 * @param preferredResources The preferred resource for this data sink.
+//	 * @return The data sink with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DataSink<T> setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+//		Preconditions.checkNotNull(minResources, "The min resources must be not null.");
+//		Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null.");
+//		Preconditions.checkArgument(minResources.isValid() && 
+//				preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources),
+//				"The values in resource must be not less than 0 and the preferred " +
+//				"resource must be greater than the min resource.");
+//
+//		this.minResources = minResources;
+//		this.preferredResources = preferredResources;
+//
+//		return this;
+//	}
+//
+//	/**
+//	 * Sets the resources for this data sink. This overrides the default resource profile.
+//	 *
+//	 * @param resources The resources for this data sink.
+//	 * @return The data sink with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DataSink<T> setResources(ResourceSpec resources) {
+//		Preconditions.checkNotNull(resources, "The resources must be not null.");
+//		Preconditions.checkArgument(resources.isValid(), "The resource values must be greater than 0.");
+//
+//		this.minResources = resources;
+//		this.preferredResources = resources;
+//
+//		return this;
+//	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
index cf0a63e..3d327e9 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
@@ -64,9 +64,9 @@ public class DeltaIteration<ST, WT> {
 	
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
 
-	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec minResources = ResourceSpec.UNKNOWN;
 
-	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec preferredResources = ResourceSpec.UNKNOWN;
 	
 	private boolean solutionSetUnManaged;
 
@@ -197,65 +197,72 @@ public class DeltaIteration<ST, WT> {
 		return parallelism;
 	}
 
-	/**
-	 * Sets the minimum and preferred resources for the iteration. This overrides the default empty resource.
-	 *	The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
-	 *
-	 * @param minResource The minimum resource for the iteration.
-	 * @param preferredResource The preferred resource for the iteration.
-	 * @return The iteration with set minimum and preferred resources.
-	 */
-	/*
-	public DeltaIteration<ST, WT> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		Preconditions.checkNotNull(minResource != null && preferredResource != null,
-				"The min and preferred resources must be not null.");
-		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
-				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
-
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
-
-		return this;
-	}*/
+//	---------------------------------------------------------------------------
+//	 Fine-grained resource profiles are an incomplete work-in-progress feature
+//	 The setters are hence commented out at this point.
+//	---------------------------------------------------------------------------
+//
+//	/**
+//	 * Sets the minimum and preferred resources for the iteration. This overrides the default empty resource.
+//	 * The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+//	 *
+//	 * @param minResources The minimum resource for the iteration.
+//	 * @param preferredResources The preferred resource for the iteration.
+//	 * @return The iteration with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DeltaIteration<ST, WT> setResource(ResourceSpec minResources, ResourceSpec preferredResources) {
+//		Preconditions.checkNotNull(minResources, "The min resources must be not null.");
+//		Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null.");
+//		Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources),
+//				"The values in resources must be not less than 0 and the preferred resources must be greater than the min resources.");
+//
+//		this.minResources = minResources;
+//		this.preferredResources = preferredResources;
+//
+//		return this;
+//	}
+//
+//	/**
+//	 * Sets the resource for the iteration, and the minimum and preferred resources are the same by default.
+//	 *	The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+//	 *
+//	 * @param resources The resource for the iteration.
+//	 * @return The iteration with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DeltaIteration<ST, WT> setResource(ResourceSpec resources) {
+//		Preconditions.checkNotNull(resources, "The resources must be not null.");
+//		Preconditions.checkArgument(resources.isValid(), "The values in resource must be not less than 0.");
+//
+//		this.minResources = resources;
+//		this.preferredResources = resources;
+//
+//		return this;
+//	}
 
 	/**
-	 * Sets the resource for the iteration, and the minimum and preferred resources are the same by default.
-	 *	The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
-	 *
-	 * @param resource The resource for the iteration.
-	 * @return The iteration with set minimum and preferred resources.
-	 */
-	/*
-	public DeltaIteration<ST, WT> setResource(ResourceSpec resource) {
-		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
-		Preconditions.checkArgument(resource.isValid(), "The values in resource must be not less than 0.");
-
-		this.minResource = resource;
-		this.preferredResource = resource;
-
-		return this;
-	}*/
-
-	/**
-	 * Gets the minimum resource from this iteration. If no minimum resource has been set,
+	 * Gets the minimum resources from this iteration. If no minimum resources have been set,
 	 * it returns the default empty resource.
 	 *
-	 * @return The minimum resource of the iteration.
+	 * @return The minimum resources of the iteration.
 	 */
-	public ResourceSpec getMinResource() {
-		return this.minResource;
+	@PublicEvolving
+	public ResourceSpec getMinResources() {
+		return this.minResources;
 	}
 
 	/**
-	 * Gets the preferred resource from this iteration. If no preferred resource has been set,
+	 * Gets the preferred resources from this iteration. If no preferred resources have been set,
 	 * it returns the default empty resource.
 	 *
-	 * @return The preferred resource of the iteration.
+	 * @return The preferred resources of the iteration.
 	 */
-	public ResourceSpec getPreferredResource() {
-		return this.preferredResource;
+	@PublicEvolving
+	public ResourceSpec getPreferredResources() {
+		return this.preferredResources;
 	}
-	
+
 	/**
 	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the
 	 * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step,

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
index 79cae14..6ae59dd 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
@@ -39,9 +39,10 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	
 	protected int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
 
-	protected ResourceSpec minResource = ResourceSpec.UNKNOWN;
+	protected ResourceSpec minResources = ResourceSpec.UNKNOWN;
+
+	protected ResourceSpec preferredResources = ResourceSpec.UNKNOWN;
 
-	protected ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
 
 	protected Operator(ExecutionEnvironment context, TypeInformation<OUT> resultType) {
 		super(context, resultType);
@@ -81,8 +82,8 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	 *
 	 * @return The minimum resource of this operator.
 	 */
-	public ResourceSpec minResource() {
-		return this.minResource;
+	public ResourceSpec getMinResources() {
+		return this.minResources;
 	}
 
 	/**
@@ -91,8 +92,8 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	 *
 	 * @return The preferred resource of this operator.
 	 */
-	public ResourceSpec preferredResource() {
-		return this.preferredResource;
+	public ResourceSpec getPreferredResources() {
+		return this.preferredResources;
 	}
 
 	/**
@@ -129,45 +130,51 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 		return returnType;
 	}
 
-	/**
-	 * Sets the minimum and preferred resources for this operator. This overrides the default empty resource.
-	 * The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
-	 *
-	 * @param minResource The minimum resource for this operator.
-	 * @param preferredResource The preferred resource for this operator.
-	 * @return The operator with set minimum and preferred resources.
-	 */
-	/*
-	public O setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		Preconditions.checkNotNull(minResource != null && preferredResource != null,
-				"The min and preferred resources must be not null.");
-		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
-				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
-
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
-
-		@SuppressWarnings("unchecked")
-		O returnType = (O) this;
-		return returnType;
-	}*/
-
-	/**
-	 * Sets the resource for this operator. This overrides the default empty minimum and preferred resources.
-	 *
-	 * @param resource The resource for this operator.
-	 * @return The operator with set minimum and preferred resources.
-	 */
-	/*
-	public O setResource(ResourceSpec resource) {
-		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
-		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
-
-		this.minResource = resource;
-		this.preferredResource = resource;
-
-		@SuppressWarnings("unchecked")
-		O returnType = (O) this;
-		return returnType;
-	}*/
+//	---------------------------------------------------------------------------
+//	 Fine-grained resource profiles are an incomplete work-in-progress feature
+//	 The setters are hence commented out at this point.
+//	---------------------------------------------------------------------------
+//
+//	/**
+//	 * Sets the minimum and preferred resources for this operator. This overrides the default empty resource.
+//	 * The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+//	 *
+//	 * @param minResources The minimum resource for this operator.
+//	 * @param preferredResources The preferred resource for this operator.
+//	 * @return The operator with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public O setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+//		Preconditions.checkNotNull(minResources, "The min resources must be not null.");
+//		Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null.");
+//
+//		Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources),
+//				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+//
+//		this.minResources = minResources;
+//		this.preferredResources = preferredResources;
+//
+//		@SuppressWarnings("unchecked")
+//		O returnType = (O) this;
+//		return returnType;
+//	}
+//
+//	/**
+//	 * Sets the resources for this operator. This overrides the default minimum and preferred resources.
+//	 *
+//	 * @param resources The resource for this operator.
+//	 * @return The operator with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public O setResources(ResourceSpec resources) {
+//		Preconditions.checkNotNull(resources, "The resource must be not null.");
+//		Preconditions.checkArgument(resources.isValid(), "The resource values must be greater than 0.");
+//
+//		this.minResources = resources;
+//		this.preferredResources = resources;
+//
+//		@SuppressWarnings("unchecked")
+//		O returnType = (O) this;
+//		return returnType;
+//	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index 909cd32..3bffd8b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -64,7 +64,7 @@ public class OperatorTranslation {
 		// translate the sink itself and connect it to the input
 		GenericDataSinkBase<T> translatedSink = sink.translateToDataFlow(input);
 
-		translatedSink.setResource(sink.getMinResource(), sink.getPreferredResource());
+		translatedSink.setResource(sink.getMinResources(), sink.getPreferredResources());
 
 		return translatedSink;
 	}
@@ -95,29 +95,29 @@ public class OperatorTranslation {
 		if (dataSet instanceof DataSource) {
 			DataSource<T> dataSource = (DataSource<T>) dataSet;
 			dataFlowOp = dataSource.translateToDataFlow();
-			dataFlowOp.setResource(dataSource.minResource(), dataSource.preferredResource());
+			dataFlowOp.setResource(dataSource.getMinResources(), dataSource.getPreferredResources());
 		}
 		else if (dataSet instanceof SingleInputOperator) {
 			SingleInputOperator<?, ?, ?> singleInputOperator = (SingleInputOperator<?, ?, ?>) dataSet;
 			dataFlowOp = translateSingleInputOperator(singleInputOperator);
-			dataFlowOp.setResource(singleInputOperator.minResource, singleInputOperator.preferredResource());
+			dataFlowOp.setResource(singleInputOperator.getMinResources(), singleInputOperator.getPreferredResources());
 		}
 		else if (dataSet instanceof TwoInputOperator) {
 			TwoInputOperator<?, ?, ?, ?> twoInputOperator = (TwoInputOperator<?, ?, ?, ?>) dataSet;
 			dataFlowOp = translateTwoInputOperator(twoInputOperator);
-			dataFlowOp.setResource(twoInputOperator.minResource(), twoInputOperator.preferredResource());
+			dataFlowOp.setResource(twoInputOperator.getMinResources(), twoInputOperator.getPreferredResources());
 		}
 		else if (dataSet instanceof BulkIterationResultSet) {
-			BulkIterationResultSet bulkIterationResultSet = (BulkIterationResultSet<?>) dataSet;
+			BulkIterationResultSet<?> bulkIterationResultSet = (BulkIterationResultSet<?>) dataSet;
 			dataFlowOp = translateBulkIteration(bulkIterationResultSet);
-			dataFlowOp.setResource(bulkIterationResultSet.getIterationHead().minResource(),
-					bulkIterationResultSet.getIterationHead().preferredResource());
+			dataFlowOp.setResource(bulkIterationResultSet.getIterationHead().getMinResources(),
+					bulkIterationResultSet.getIterationHead().getPreferredResources());
 		}
 		else if (dataSet instanceof DeltaIterationResultSet) {
-			DeltaIterationResultSet deltaIterationResultSet = (DeltaIterationResultSet<?, ?>) dataSet;
+			DeltaIterationResultSet<?, ?> deltaIterationResultSet = (DeltaIterationResultSet<?, ?>) dataSet;
 			dataFlowOp = translateDeltaIteration(deltaIterationResultSet);
-			dataFlowOp.setResource(deltaIterationResultSet.getIterationHead().getMinResource(),
-					deltaIterationResultSet.getIterationHead().getPreferredResource());
+			dataFlowOp.setResource(deltaIterationResultSet.getIterationHead().getMinResources(),
+					deltaIterationResultSet.getIterationHead().getPreferredResources());
 		}
 		else if (dataSet instanceof DeltaIteration.SolutionSetPlaceHolder || dataSet instanceof DeltaIteration.WorksetPlaceHolder) {
 			throw new InvalidProgramException("A data set that is part of a delta iteration was used as a sink or action."

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
index 4ef91b3..723c532 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
@@ -310,12 +310,12 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 		return this.parallelism;
 	}
 
-	public ResourceSpec getMinResource() {
-		return this.template.getOperator().getMinResource();
+	public ResourceSpec getMinResources() {
+		return this.template.getOperator().getMinResources();
 	}
 
-	public ResourceSpec getPreferredResource() {
-		return this.template.getOperator().getPreferredResource();
+	public ResourceSpec getPreferredResources() {
+		return this.template.getOperator().getPreferredResources();
 	}
 	
 	public long getGuaranteedAvailableMemory() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 5cfb601..bfe7567 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -177,48 +177,55 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
         "parallelism.")
   }
 
-  /**
-   * Sets the minimum and preferred resources of this operation.
-   */
-  /*
-  def resource(minResource: ResourceSpec, preferredResource: ResourceSpec) : Unit = {
-    javaSet match {
-      case ds: DataSource[_] => ds.setResource(minResource, preferredResource)
-      case op: Operator[_, _] => op.setResource(minResource, preferredResource)
-      case di: DeltaIterationResultSet[_, _] =>
-        di.getIterationHead.setResource(minResource, preferredResource)
-      case _ =>
-        throw new UnsupportedOperationException("Operator does not support " +
-          "configuring custom resources specs.")
-    }
-    this
-  }*/
 
-  /**
-   * Sets the resource of this operation.
-   */
-  /*
-  def resource(resource: ResourceSpec) : Unit = {
-    this.resource(resource, resource)
-  }*/
+// ---------------------------------------------------------------------------
+//  Fine-grained resource profiles are an incomplete work-in-progress feature
+//  The setters are hence commented out at this point.
+// ---------------------------------------------------------------------------
+//  /**
+//   * Sets the minimum and preferred resources of this operation.
+//   */
+//  @PublicEvolving
+//  def resources(minResources: ResourceSpec, preferredResources: ResourceSpec) : Unit = {
+//    javaSet match {
+//      case ds: DataSource[_] => ds.setResources(minResources, preferredResources)
+//      case op: Operator[_, _] => op.setResources(minResources, preferredResources)
+//      case di: DeltaIterationResultSet[_, _] =>
+//        di.getIterationHead.setResources(minResources, preferredResources)
+//      case _ =>
+//        throw new UnsupportedOperationException("Operator does not support " +
+//          "configuring custom resources specs.")
+//    }
+//    this
+//  }
+//
+//  /**
+//   * Sets the resource of this operation.
+//   */
+//  @PublicEvolving
+//  def resources(resources: ResourceSpec) : Unit = {
+//    this.resources(resources, resources)
+//  }
 
   /**
-   * Returns the minimum resource of this operation.
+   * Returns the minimum resources of this operation.
    */
-  def minResource: ResourceSpec = javaSet match {
-    case ds: DataSource[_] => ds.minResource()
-    case op: Operator[_, _] => op.minResource
+  @PublicEvolving
+  def minResources: ResourceSpec = javaSet match {
+    case ds: DataSource[_] => ds.getMinResources()
+    case op: Operator[_, _] => op.getMinResources()
     case _ =>
       throw new UnsupportedOperationException("Operator does not support " +
         "configuring custom resources specs.")
   }
 
   /**
-   * Returns the preferred resource of this operation.
+   * Returns the preferred resources of this operation.
    */
-  def preferredResource: ResourceSpec = javaSet match {
-    case ds: DataSource[_] => ds.preferredResource()
-    case op: Operator[_, _] => op.preferredResource
+  @PublicEvolving
+  def preferredResources: ResourceSpec = javaSet match {
+    case ds: DataSource[_] => ds.getPreferredResources()
+    case op: Operator[_, _] => op.getPreferredResources()
     case _ =>
       throw new UnsupportedOperationException("Operator does not support " +
         "configuring custom resources specs.")

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index ae1c39a..c443758 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -144,21 +144,23 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Gets the minimum resource for this operator.
+	 * Gets the minimum resources for this operator.
 	 *
-	 * @return The minimum resource set for this operator.
+	 * @return The minimum resources set for this operator.
 	 */
-	public ResourceSpec minResource() {
-		return transformation.getMinResource();
+	@PublicEvolving
+	public ResourceSpec getMinResources() {
+		return transformation.getMinResources();
 	}
 
 	/**
-	 * Gets the preferred resource for this operator.
+	 * Gets the preferred resources for this operator.
 	 *
-	 * @return The preferred resource set for this operator.
+	 * @return The preferred resources set for this operator.
 	 */
-	public ResourceSpec preferredResource() {
-		return transformation.getPreferredResource();
+	@PublicEvolving
+	public ResourceSpec getPreferredResources() {
+		return transformation.getPreferredResources();
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index 69e21d6..39d81c6 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.transformations.SinkTransformation;
 @Public
 public class DataStreamSink<T> {
 
-	SinkTransformation<T> transformation;
+	private final SinkTransformation<T> transformation;
 
 	@SuppressWarnings("unchecked")
 	protected DataStreamSink(DataStream<T> inputStream, StreamSink<T> operator) {
@@ -113,41 +113,45 @@ public class DataStreamSink<T> {
 		return this;
 	}
 
-	/**
-	 * Sets the minimum and preferred resources for this sink, and the lower and upper resource limits will
-	 * be considered in resource resize feature for future plan.
-	 *
-	 * @param minResource The minimum resource for this sink.
-	 * @param preferredResource The preferred resource for this sink
-	 * @return The sink with set minimum and preferred resources.
-	 */
-	/*
-	public DataStreamSink<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		Preconditions.checkNotNull(minResource != null && preferredResource != null,
-				"The min and preferred resources must be not null.");
-		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
-				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
-
-		transformation.setResource(minResource, preferredResource);
-
-		return this;
-	}*/
-
-	/**
-	 * Sets the resource for this sink, the minimum and preferred resources are the same by default.
-	 *
-	 * @param resource The resource for this sink.
-	 * @return The sink with set minimum and preferred resources.
-	 */
-	/*
-	public DataStreamSink<T> setResource(ResourceSpec resource) {
-		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
-		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
-
-		transformation.setResource(resource, resource);
-
-		return this;
-	}*/
+//	---------------------------------------------------------------------------
+//	 Fine-grained resource profiles are an incomplete work-in-progress feature
+//	 The setters are hence commented out at this point.
+//	---------------------------------------------------------------------------
+//	/**
+//	 * Sets the minimum and preferred resources for this sink, and the lower and upper resource limits will
+//	 * be considered in resource resize feature for future plan.
+//	 *
+//	 * @param minResources The minimum resources for this sink.
+//	 * @param preferredResources The preferred resources for this sink
+//	 * @return The sink with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DataStreamSink<T> setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+//		Preconditions.checkNotNull(minResources, "The min resources must be not null.");
+//		Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null.");
+//		Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources),
+//				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+//
+//		transformation.setResources(minResources, preferredResources);
+//
+//		return this;
+//	}
+//
+//	/**
+//	 * Sets the resource for this sink, the minimum and preferred resources are the same by default.
+//	 *
+//	 * @param resources The resource for this sink.
+//	 * @return The sink with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public DataStreamSink<T> setResources(ResourceSpec resources) {
+//		Preconditions.checkNotNull(resources, "The resource must be not null.");
+//		Preconditions.checkArgument(resources.isValid(), "The resource values must be greater than 0.");
+//
+//		transformation.setResources(resources, resources);
+//
+//		return this;
+//	}
 
 	/**
 	 * Turns off chaining for this operator so thread co-location will not be

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index d856603..859c6d5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -154,41 +154,45 @@ public class SingleOutputStreamOperator<T> extends DataStream<T> {
 		return this;
 	}
 
-	/**
-	 * Sets the minimum and preferred resources for this operator, and the lower and upper resource limits will
-	 * be considered in dynamic resource resize feature for future plan.
-	 *
-	 * @param minResource The minimum resource for this operator.
-	 * @param preferredResource The preferred resource for this operator.
-	 * @return The operator with set minimum and preferred resources.
-	 */
-	/*
-	public SingleOutputStreamOperator<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		Preconditions.checkArgument(minResource != null && preferredResource != null,
-				"The min and preferred resources must be not null.");
-		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
-				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
-
-		transformation.setResource(minResource, preferredResource);
-
-		return this;
-	}*/
-
-	/**
-	 * Sets the resource for this operator, the minimum and preferred resources are the same by default.
-	 *
-	 * @param resource The resource for this operator.
-	 * @return The operator with set minimum and preferred resources.
-	 */
-	/*
-	public SingleOutputStreamOperator<T> setResource(ResourceSpec resource) {
-		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
-		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
-
-		transformation.setResource(resource, resource);
-
-		return this;
-	}*/
+//	---------------------------------------------------------------------------
+//	 Fine-grained resource profiles are an incomplete work-in-progress feature
+//	 The setters are hence commented out at this point.
+//	---------------------------------------------------------------------------
+//	/**
+//	 * Sets the minimum and preferred resources for this operator, and the lower and upper resource limits will
+//	 * be considered in dynamic resource resize feature for future plan.
+//	 *
+//	 * @param minResources The minimum resources for this operator.
+//	 * @param preferredResources The preferred resources for this operator.
+//	 * @return The operator with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public SingleOutputStreamOperator<T> setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+//		Preconditions.checkNotNull(minResources, "The min resources must be not null.");
+//		Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null.");
+//		Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResource.lessThanOrEqual(preferredResources),
+//				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+//
+//		transformation.setResources(minResources, preferredResources);
+//
+//		return this;
+//	}
+//
+//	/**
+//	 * Sets the resources for this operator, the minimum and preferred resources are the same by default.
+//	 *
+//	 * @param resources The resources for this operator.
+//	 * @return The operator with set minimum and preferred resources.
+//	 */
+//	@PublicEvolving
+//	public SingleOutputStreamOperator<T> setResources(ResourceSpec resources) {
+//		Preconditions.checkNotNull(resources, "The resource must be not null.");
+//		Preconditions.checkArgument(resources.isValid(), "The resource values must be greater than 0.");
+//
+//		transformation.setResources(resources, resources);
+//
+//		return this;
+//	}
 
 	private boolean canBeParallel() {
 		return !nonParallel;

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index fcbc607..a87e63d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -416,7 +416,7 @@ public class StreamGraph extends StreamingPlan {
 
 	public void setResource(int vertexID, ResourceSpec minResource, ResourceSpec preferredResource) {
 		if (getStreamNode(vertexID) != null) {
-			getStreamNode(vertexID).setResource(minResource, preferredResource);
+			getStreamNode(vertexID).setResources(minResource, preferredResource);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
index af92421..f55ff47 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -202,8 +202,8 @@ public class StreamGraphGenerator {
 			streamGraph.setTransformationUserHash(transform.getId(), transform.getUserProvidedNodeHash());
 		}
 
-		if (transform.getMinResource() != null && transform.getPreferredResource() != null) {
-			streamGraph.setResource(transform.getId(), transform.getMinResource(), transform.getPreferredResource());
+		if (transform.getMinResources() != null && transform.getPreferredResources() != null) {
+			streamGraph.setResource(transform.getId(), transform.getMinResources(), transform.getPreferredResources());
 		}
 
 		return transformedIds;

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
index 0bf9adf..2d2e1e75 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
@@ -49,8 +49,8 @@ public class StreamNode implements Serializable {
 	 * dynamic scaling and the number of key groups used for partitioned state.
 	 */
 	private int maxParallelism;
-	private ResourceSpec minResource;
-	private ResourceSpec preferredResource;
+	private ResourceSpec minResources;
+	private ResourceSpec preferredResources;
 	private Long bufferTimeout = null;
 	private final String operatorName;
 	private String slotSharingGroup;
@@ -168,17 +168,17 @@ public class StreamNode implements Serializable {
 		this.maxParallelism = maxParallelism;
 	}
 
-	public ResourceSpec getMinResource() {
-		return minResource;
+	public ResourceSpec getMinResources() {
+		return minResources;
 	}
 
-	public ResourceSpec getPreferredResource() {
-		return preferredResource;
+	public ResourceSpec getPreferredResources() {
+		return preferredResources;
 	}
 
-	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
+	public void setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+		this.minResources = minResources;
+		this.preferredResources = preferredResources;
 	}
 
 	public Long getBufferTimeout() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
index 1d22454..24b5736 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
@@ -30,6 +30,8 @@ import org.apache.flink.util.Preconditions;
 
 import java.util.Collection;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * A {@code StreamTransformation} represents the operation that creates a
  * {@link org.apache.flink.streaming.api.datastream.DataStream}. Every
@@ -127,16 +129,16 @@ public abstract class StreamTransformation<T> {
 	private int maxParallelism = -1;
 
 	/**
-	 *  The minimum resource for this stream transformation. It defines the lower limit for
-	 *  dynamic resource resize in future plan.
+	 *  The minimum resources for this stream transformation. It defines the lower limit for
+	 *  dynamic resources resize in future plan.
 	 */
-	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec minResources = ResourceSpec.UNKNOWN;
 
 	/**
-	 *  The preferred resource for this stream transformation. It defines the upper limit for
+	 *  The preferred resources for this stream transformation. It defines the upper limit for
 	 *  dynamic resource resize in future plan.
 	 */
-	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+	private ResourceSpec preferredResources = ResourceSpec.UNKNOWN;
 
 	/**
 	 * User-specified ID for this transformation. This is used to assign the
@@ -229,12 +231,12 @@ public abstract class StreamTransformation<T> {
 	/**
 	 * Sets the minimum and preferred resources for this stream transformation.
 	 *
-	 * @param minResource The minimum resource of this transformation.
-	 * @param preferredResource The preferred resource of this transformation.
+	 * @param minResources The minimum resource of this transformation.
+	 * @param preferredResources The preferred resource of this transformation.
 	 */
-	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
-		this.minResource = minResource;
-		this.preferredResource = preferredResource;
+	public void setResources(ResourceSpec minResources, ResourceSpec preferredResources) {
+		this.minResources = checkNotNull(minResources);
+		this.preferredResources = checkNotNull(preferredResources);
 	}
 
 	/**
@@ -242,8 +244,8 @@ public abstract class StreamTransformation<T> {
 	 *
 	 * @return The minimum resource of this transformation.
 	 */
-	public ResourceSpec getMinResource() {
-		return minResource;
+	public ResourceSpec getMinResources() {
+		return minResources;
 	}
 
 	/**
@@ -251,8 +253,8 @@ public abstract class StreamTransformation<T> {
 	 *
 	 * @return The preferred resource of this transformation.
 	 */
-	public ResourceSpec getPreferredResource() {
-		return preferredResource;
+	public ResourceSpec getPreferredResources() {
+		return preferredResources;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index e42fb3f..35e1f23 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -146,36 +146,42 @@ class DataStream[T](stream: JavaStream[T]) {
   }
 
   /**
-   * Returns the minimum resource of this operation.
+   * Returns the minimum resources of this operation.
    */
-  def minResource: ResourceSpec = stream.minResource()
-
-  /**
-   * Returns the preferred resource of this operation.
-   */
-  def preferredResource: ResourceSpec = stream.preferredResource()
-
-  /**
-   * Sets the minimum and preferred resources of this operation.
-   */
-  /*
-  def resource(minResource: ResourceSpec, preferredResource: ResourceSpec) : DataStream[T] =
-    stream match {
-      case stream : SingleOutputStreamOperator[T] => asScalaStream(stream.setResource(
-        minResource, preferredResource))
-      case _ =>
-        throw new UnsupportedOperationException("Operator does not support " +
-          "configuring custom resources specs.")
-      this
-  }*/
+  @PublicEvolving
+  def minResources: ResourceSpec = stream.getMinResources()
 
   /**
-   * Sets the resource of this operation.
+   * Returns the preferred resources of this operation.
    */
-  /*
-  def resource(resource: ResourceSpec) : Unit = {
-    this.resource(resource, resource)
-  }*/
+  @PublicEvolving
+  def preferredResources: ResourceSpec = stream.getPreferredResources()
+
+// ---------------------------------------------------------------------------
+//  Fine-grained resource profiles are an incomplete work-in-progress feature
+//  The setters are hence commented out at this point.
+// ---------------------------------------------------------------------------
+//  /**
+//   * Sets the minimum and preferred resources of this operation.
+//   */
+//  @PublicEvolving
+//  def resources(minResources: ResourceSpec, preferredResources: ResourceSpec) : DataStream[T] =
+//    stream match {
+//      case stream : SingleOutputStreamOperator[T] => asScalaStream(
+//        stream.setResources(minResources, preferredResources))
+//      case _ =>
+//        throw new UnsupportedOperationException("Operator does not support " +
+//          "configuring custom resources specs.")
+//      this
+//  }
+//
+//  /**
+//   * Sets the resource of this operation.
+//   */
+//  @PublicEvolving
+//  def resources(resources: ResourceSpec) : Unit = {
+//    this.resources(resources, resources)
+//  }
 
   /**
    * Gets the name of the current data stream. This name is

http://git-wip-us.apache.org/repos/asf/flink/blob/9912de21/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala
index 907ad9f..7abb392 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/ScalaAPICompletenessTestBase.scala
@@ -41,21 +41,6 @@ abstract class ScalaAPICompletenessTestBase extends TestLogger {
   protected def isExcludedByName(method: Method): Boolean
 
   /**
-   * Determines whether a method is excluded by an interface it uses.
-   */
-  protected def isExcludedByInterface(method: Method): Boolean = {
-    val excludedInterfaces =
-      Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil")
-    def toComparisionKey(method: Method) =
-      (method.getReturnType, method.getName, method.getGenericReturnType)
-    val interfaces = method.getDeclaringClass.getInterfaces.filter { i =>
-      excludedInterfaces.contains(i.getName)
-    }
-    val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey))
-    excludedMethods.contains(toComparisionKey(method))
-  }
-
-  /**
    * Utility to be called during the test.
    */
   protected def checkMethods(
@@ -66,26 +51,33 @@ abstract class ScalaAPICompletenessTestBase extends TestLogger {
     val javaMethods = javaClass.getMethods
       .filterNot(_.isAccessible)
       .filterNot(isExcludedByName)
-      .filterNot(isExcludedByInterface)
       .map(m => m.getName).toSet
 
     val scalaMethods = scalaClass.getMethods
       .filterNot(_.isAccessible)
       .filterNot(isExcludedByName)
-      .filterNot(isExcludedByInterface)
       .map(m => m.getName).toSet
 
     val missingMethods = javaMethods -- scalaMethods
 
-    for (method <- missingMethods) {
-      fail("Method " + method + " from " + javaClass + " is missing from " + scalaClassName + ".")
+    for (javaMethod <- missingMethods) {
+      // check if the method simply follows different getter / setter conventions in Scala / Java
+      // for example Java: getFoo() should match Scala: foo()
+      if (!containsScalaGetterLike(javaMethod, scalaMethods)) {
+        fail(s"Method $javaMethod from $javaClass is missing from $scalaClassName.")
+      }
     }
   }
 
-  protected def checkEquality(scalaInstance: AnyRef, extractJavaFun : ((AnyRef) => AnyRef)) {
-    val javaInstance = extractJavaFun(scalaInstance)
+  protected def containsScalaGetterLike(javaMethod: String, scalaMethods: Set[String]): Boolean = {
+    if (javaMethod.startsWith("get") && javaMethod.length >= 4) {
+      val scalaMethodName = Character.toLowerCase(javaMethod.charAt(3)) + javaMethod.substring(4)
+      scalaMethods.contains(scalaMethodName)
+    } else {
+      false
+    }
   }
-
+  
   /**
    * Tests to be performed to ensure API completeness.
    */


[05/11] flink git commit: [FLINK-5501] [runtime] Followups and improvements to RunningJobsRegistry

Posted by se...@apache.org.
[FLINK-5501] [runtime] Followups and improvements to RunningJobsRegistry

This commit changes the following:
  - Remove the unsafe 'isJobRunning()' method.
  - Exctract duplicate code into utility functions
  - Simplify the NonHaRegistry by using a map rather than two sets
  - Improve exception handling / error messages for the ZooKeeper-based registry
  - Slight improvement of error handling in the JobManagerRunner
  - Compare enums with '==' (better null-pointer safety)
  - Correct 'expected' and 'actual' parameters in 'assertEquals'
  - Forward tests also to the HDFS file based registry test


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

Branch: refs/heads/master
Commit: e0dede9fb0a2ef7560254b6fc40d852ebf16c956
Parents: e7011d7
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Feb 27 21:56:36 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../configuration/HighAvailabilityOptions.java  |  4 ++
 .../FsNegativeRunningJobsRegistryTest.java      | 40 +++++++----
 .../FsNegativeRunningJobsRegistry.java          | 68 +++++++-----------
 .../highavailability/RunningJobsRegistry.java   | 39 ++++++-----
 .../highavailability/ZookeeperRegistry.java     | 73 ++++++++++----------
 .../highavailability/nonha/NonHaRegistry.java   | 45 ++++--------
 .../runtime/jobmaster/JobManagerRunner.java     | 20 +++---
 .../minicluster/MiniClusterJobDispatcher.java   |  2 +-
 .../resourcemanager/JobLeaderIdService.java     |  4 +-
 .../FsNegativeRunningJobsRegistryTest.java      | 37 +++++-----
 .../highavailability/ZooKeeperRegistryTest.java | 33 ++++-----
 11 files changed, 166 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
index 4792eba..b883bc3 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
@@ -124,6 +124,10 @@ public class HighAvailabilityOptions {
 			.defaultValue(3)
 			.withDeprecatedKeys("recovery.zookeeper.client.max-retry-attempts");
 
+	public static final ConfigOption<String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH = 
+			key("high-availability.zookeeper.job.registry")
+			.defaultValue("/running_job_registry/");
+
 	// ------------------------------------------------------------------------
 
 	/** Not intended to be instantiated */

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
index 40d75e8..bb27b8b 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.highavailability.FsNegativeRunningJobsRegistry;
 
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 
@@ -33,9 +34,11 @@ import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
+/**
+ * Tests for the {@link FsNegativeRunningJobsRegistry} on HDFS.
+ */
 public class FsNegativeRunningJobsRegistryTest {
 
 	@ClassRule
@@ -83,20 +86,22 @@ public class FsNegativeRunningJobsRegistryTest {
 
 		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(workDir);
 
-		// initially, without any call, the job is considered running
-		assertTrue(registry.isJobRunning(jid));
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir);
+
+		// initially, without any call, the job is pending
+		assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid));
 
-		// repeated setting should not affect the status
+		// after set running, the job is running
 		registry.setJobRunning(jid);
-		assertTrue(registry.isJobRunning(jid));
+		assertEquals(JobSchedulingStatus.RUNNING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.RUNNING, otherRegistry.getJobSchedulingStatus(jid));
 
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
-		assertFalse(registry.isJobRunning(jid));
-
-		// another registry should pick this up
-		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir);
-		assertFalse(otherRegistry.isJobRunning(jid));
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid));
 	}
 
 	@Test
@@ -108,14 +113,19 @@ public class FsNegativeRunningJobsRegistryTest {
 
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
-		assertFalse(registry.isJobRunning(jid));
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
 
-		// set the job to back to running and validate
+		// set the job to running does not overwrite the finished status
 		registry.setJobRunning(jid);
-		assertTrue(registry.isJobRunning(jid));
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
 
 		// another registry should pick this up
 		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir);
-		assertTrue(otherRegistry.isJobRunning(jid));
+		assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid));
+
+		// clear the running and finished marker, it will be pending
+		otherRegistry.clearJob(jid);
+		assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
index 9e92263..cb79a65 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.highavailability;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
 
 import java.io.FileNotFoundException;
@@ -30,19 +31,18 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * This {@link RunningJobsRegistry} tracks the status jobs via marker files,
- * marking running jobs via running marker files,
- * marking finished jobs via finished marker files.
+ * marking running jobs viarunning marker files, marking finished jobs via finished marker files.
  * 
  * <p>The general contract is the following:
  * <ul>
  *     <li>Initially, a marker file does not exist (no one created it, yet), which means
- *         the specific job is pending</li>
+ *         the specific job is pending.</li>
  *     <li>The first JobManager that granted leadership calls this service to create the running marker file,
  *         which marks the job as running.</li>
+ *     <li>If a JobManager gains leadership but sees the running marker file,
+ *         it will realize that the job has been scheduled already and needs reconciling.</li>
  *     <li>The JobManager that finishes calls this service to create the marker file,
  *         which marks the job as finished.</li>
- *     <li>If a JobManager gains leadership but see the running marker file,
- *         it will realize that the job has been scheduled and need reconciling.</li>
  *     <li>If a JobManager gains leadership at some point when shutdown is in progress,
  *         it will see the marker file and realize that the job is finished.</li>
  *     <li>The application framework is expected to clean the file once the application
@@ -50,7 +50,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *         start the job, even if it gains leadership.</li>
  * </ul>
  * 
- * <p>It is especially tailored towards deployment modes like for example
+ * <p>This registry is especially tailored towards deployment modes like for example
  * YARN, where HDFS is available as a persistent file system, and the YARN
  * application's working directories on HDFS are automatically cleaned
  * up after the application completed. 
@@ -99,8 +99,8 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 		// to be safe, attempt to write to the working directory, to
 		// catch problems early
 		final Path testFile = new Path(workingDirectory, ".registry_test");
-		try (FSDataOutputStream out = fileSystem.create(testFile, false)) {
-			out.write(42);
+		try {
+			createFile(testFile, false);
 		}
 		catch (IOException e) {
 			throw new IOException("Unable to write to working directory: " + workingDirectory, e);
@@ -119,9 +119,7 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 
 		// create the file
 		// to avoid an exception if the job already exists, set overwrite=true
-		try (FSDataOutputStream out = fileSystem.create(filePath, true)) {
-			out.write(42);
-		}
+		createFile(filePath, true);
 	}
 
 	@Override
@@ -131,25 +129,7 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 
 		// create the file
 		// to avoid an exception if the job already exists, set overwrite=true
-		try (FSDataOutputStream out = fileSystem.create(filePath, true)) {
-			out.write(42);
-		}
-	}
-
-	@Override
-	public boolean isJobRunning(JobID jobID) throws IOException {
-		checkNotNull(jobID, "jobID");
-
-		// check for the existence of the file
-		try {
-			fileSystem.getFileStatus(createMarkerFilePath(RUNNING_PREFIX, jobID));
-			// file was found --> job is running
-			return true;
-		}
-		catch (FileNotFoundException e) {
-			// file does not exist, job is not running
-			return false;
-		}
+		createFile(filePath, true);
 	}
 
 	@Override
@@ -157,21 +137,16 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 		checkNotNull(jobID, "jobID");
 
 		// first check for the existence of the complete file
-		try {
-			fileSystem.getFileStatus(createMarkerFilePath(DONE_PREFIX, jobID));
+		if (fileSystem.exists(createMarkerFilePath(DONE_PREFIX, jobID))) {
 			// complete file was found --> job is terminated
 			return JobSchedulingStatus.DONE;
 		}
-		catch (FileNotFoundException e) {
-			// file does not exist, job is running or pending
-		}
 		// check for the existence of the running file
-		try {
-			fileSystem.getFileStatus(createMarkerFilePath(RUNNING_PREFIX, jobID));
+		else if (fileSystem.exists(createMarkerFilePath(RUNNING_PREFIX, jobID))) {
 			// running file was found --> job is terminated
 			return JobSchedulingStatus.RUNNING;
 		}
-		catch (FileNotFoundException e) {
+		else {
 			// file does not exist, job is not scheduled
 			return JobSchedulingStatus.PENDING;
 		}
@@ -181,25 +156,30 @@ public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
 	public void clearJob(JobID jobID) throws IOException {
 		checkNotNull(jobID, "jobID");
 		final Path runningFilePath = createMarkerFilePath(RUNNING_PREFIX, jobID);
+		final Path doneFilePath = createMarkerFilePath(DONE_PREFIX, jobID);
 
 		// delete the running marker file, if it exists
 		try {
 			fileSystem.delete(runningFilePath, false);
 		}
-		catch (FileNotFoundException e) {
-		}
-
-		final Path doneFilePath = createMarkerFilePath(DONE_PREFIX, jobID);
+		catch (FileNotFoundException ignored) {}
 
 		// delete the finished marker file, if it exists
 		try {
 			fileSystem.delete(doneFilePath, false);
 		}
-		catch (FileNotFoundException e) {
-		}
+		catch (FileNotFoundException ignored) {}
 	}
 
 	private Path createMarkerFilePath(String prefix, JobID jobId) {
 		return new Path(basePath, prefix + jobId.toString());
 	}
+
+	private void createFile(Path path, boolean overwrite) throws IOException {
+		final WriteMode writeMode = overwrite ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE;
+
+		try (FSDataOutputStream out = fileSystem.create(path, writeMode)) {
+			out.write(42);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
index 020f2ca..43e5ac5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
@@ -23,29 +23,40 @@ import org.apache.flink.api.common.JobID;
 import java.io.IOException;
 
 /**
- * This registry tracks if a certain job is running.
+ * A simple registry that tracks if a certain job is pending execution, running, or completed.
  * 
  * <p>This registry is used in highly-available setups with multiple master nodes,
  * to determine whether a new leader should attempt to recover a certain job (because the 
  * job is still running), or whether the job has already finished successfully (in case of a
  * finite job) and the leader has only been granted leadership because the previous leader
  * quit cleanly after the job was finished.
+ * 
+ * <p>In addition, the registry can help to determine whether a newly assigned leader JobManager
+ * should attempt reconciliation with running TaskManagers, or immediately schedule the job from
+ * the latest checkpoint/savepoint. 
  */
 public interface RunningJobsRegistry {
 
-	public enum JobSchedulingStatus {
-		/** Job has not been scheduled */
+	/**
+	 * The scheduling status of a job, as maintained by the {@code RunningJobsRegistry}.
+	 */
+	enum JobSchedulingStatus {
+
+		/** Job has not been scheduled, yet */
 		PENDING,
 
-		/** Job has been scheduled */
+		/** Job has been scheduled and is not yet finished */
 		RUNNING,
 
-		/** Job has been finished */
+		/** Job has been finished, successfully or unsuccessfully */
 		DONE;
 	}
 
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Marks a job as running.
+	 * Marks a job as running. Requesting the job's status via the {@link #getJobSchedulingStatus(JobID)}
+	 * method will return {@link JobSchedulingStatus#RUNNING}.
 	 * 
 	 * @param jobID The id of the job.
 	 *
@@ -55,7 +66,8 @@ public interface RunningJobsRegistry {
 	void setJobRunning(JobID jobID) throws IOException;
 
 	/**
-	 * Marks a job as running.
+	 * Marks a job as completed. Requesting the job's status via the {@link #getJobSchedulingStatus(JobID)}
+	 * method will return {@link JobSchedulingStatus#DONE}.
 	 *
 	 * @param jobID The id of the job.
 	 * 
@@ -65,18 +77,7 @@ public interface RunningJobsRegistry {
 	void setJobFinished(JobID jobID) throws IOException;
 
 	/**
-	 * Checks whether a job is running.
-	 *
-	 * @param jobID The id of the job to check.
-	 * @return True if the job is still running, false otherwise.
-	 * 
-	 * @throws IOException Thrown when the communication with the highly-available storage or registry
-	 *                     failed and could not be retried.
-	 */
-	boolean isJobRunning(JobID jobID) throws IOException;
-
-	/**
-	 * Get the scheduing status of a job.
+	 * Gets the scheduling status of a job.
 	 *
 	 * @param jobID The id of the job to check.
 	 * @return The job scheduling status.

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
index 31a4535..a8be35a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
@@ -26,6 +26,7 @@ import org.apache.zookeeper.data.Stat;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.Arrays;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -33,20 +34,17 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * A zookeeper based registry for running jobs, highly available.
  */
 public class ZookeeperRegistry implements RunningJobsRegistry {
-	
-	private static final String DEFAULT_HA_JOB_REGISTRY_PATH = "/running_job_registry/";
+
+	private static final Charset ENCODING = Charset.forName("utf-8");
 
 	/** The ZooKeeper client to use */
 	private final CuratorFramework client;
 
 	private final String runningJobPath;
 
-	private static final String HA_JOB_REGISTRY_PATH = "high-availability.zookeeper.job.registry";
-
 	public ZookeeperRegistry(final CuratorFramework client, final Configuration configuration) {
-		this.client = client;
-		runningJobPath = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT) + 
-			configuration.getString(HA_JOB_REGISTRY_PATH, DEFAULT_HA_JOB_REGISTRY_PATH);
+		this.client = checkNotNull(client, "client");
+		this.runningJobPath = configuration.getString(HighAvailabilityOptions.ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH);
 	}
 
 	@Override
@@ -54,12 +52,10 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		checkNotNull(jobID);
 
 		try {
-			String zkPath = runningJobPath + jobID.toString();
-			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-			this.client.setData().forPath(zkPath, JobSchedulingStatus.RUNNING.name().getBytes(Charset.forName("utf-8")));
+			writeEnumToZooKeeper(jobID, JobSchedulingStatus.RUNNING);
 		}
 		catch (Exception e) {
-			throw new IOException("Set running state to zk fail for job " + jobID.toString(), e);
+			throw new IOException("Failed to set RUNNING state in ZooKeeper for job " + jobID, e);
 		}
 	}
 
@@ -68,44 +64,36 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		checkNotNull(jobID);
 
 		try {
-			String zkPath = runningJobPath + jobID.toString();
-			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-			this.client.setData().forPath(zkPath, JobSchedulingStatus.DONE.name().getBytes(Charset.forName("utf-8")));
+			writeEnumToZooKeeper(jobID, JobSchedulingStatus.DONE);
 		}
 		catch (Exception e) {
-			throw new IOException("Set finished state to zk fail for job " + jobID.toString(), e);
+			throw new IOException("Failed to set DONE state in ZooKeeper for job " + jobID, e);
 		}
 	}
 
 	@Override
-	public boolean isJobRunning(JobID jobID) throws IOException {
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
 		checkNotNull(jobID);
 
 		try {
-			Stat stat = client.checkExists().forPath(runningJobPath + jobID.toString());
+			final String zkPath = createZkPath(jobID);
+			final Stat stat = client.checkExists().forPath(zkPath);
 			if (stat != null) {
-				byte[] data = client.getData().forPath(runningJobPath + jobID.toString());
-				if (JobSchedulingStatus.RUNNING.name().equals(new String(data))) {
-					return true;
+				// found some data, try to parse it
+				final byte[] data = client.getData().forPath(zkPath);
+				if (data != null) {
+					try {
+						final String name = new String(data, ENCODING);
+						return JobSchedulingStatus.valueOf(name);
+					}
+					catch (IllegalArgumentException e) {
+						throw new IOException("Found corrupt data in ZooKeeper: " + 
+								Arrays.toString(data) + " is no valid job status");
+					}
 				}
 			}
-			return false;
-		}
-		catch (Exception e) {
-			throw new IOException("Get running state from zk fail for job " + jobID.toString(), e);
-		}
-	}
 
-	@Override
-	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
-		checkNotNull(jobID);
-
-		try {
-			Stat stat = client.checkExists().forPath(runningJobPath + jobID.toString());
-			if (stat != null) {
-				byte[] data = client.getData().forPath(runningJobPath + jobID.toString());
-				return JobSchedulingStatus.valueOf(new String(data));
-			}
+			// nothing found, yet, must be in status 'PENDING'
 			return JobSchedulingStatus.PENDING;
 		}
 		catch (Exception e) {
@@ -118,13 +106,22 @@ public class ZookeeperRegistry implements RunningJobsRegistry {
 		checkNotNull(jobID);
 
 		try {
-			String zkPath = runningJobPath + jobID.toString();
+			final String zkPath = createZkPath(jobID);
 			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
 			this.client.delete().forPath(zkPath);
 		}
 		catch (Exception e) {
-			throw new IOException("Clear job state from zk fail for " + jobID.toString(), e);
+			throw new IOException("Failed to clear job state from ZooKeeper for job " + jobID, e);
 		}
 	}
 
+	private String createZkPath(JobID jobID) {
+		return runningJobPath + jobID.toString();
+	}
+
+	private void writeEnumToZooKeeper(JobID jobID, JobSchedulingStatus status) throws Exception {
+		final String zkPath = createZkPath(jobID);
+		this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
+		this.client.setData().forPath(zkPath, status.name().getBytes(ENCODING));
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
index e331212..ab1ce47 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
@@ -20,9 +20,8 @@ package org.apache.flink.runtime.highavailability.nonha;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
-import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 
-import java.util.HashSet;
+import java.util.HashMap;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -32,18 +31,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 public class NonHaRegistry implements RunningJobsRegistry {
 
 	/** The currently running jobs */
-	private final HashSet<JobID> running = new HashSet<>();
-
-	/** The currently finished jobs */
-	private final HashSet<JobID> finished = new HashSet<>();
+	private final HashMap<JobID, JobSchedulingStatus> jobStatus = new HashMap<>();
 
 	@Override
 	public void setJobRunning(JobID jobID) {
 		checkNotNull(jobID);
 
-		synchronized (running) {
-			running.add(jobID);
-			finished.remove(jobID);
+		synchronized (jobStatus) {
+			jobStatus.put(jobID, JobSchedulingStatus.RUNNING);
 		}
 	}
 
@@ -51,33 +46,18 @@ public class NonHaRegistry implements RunningJobsRegistry {
 	public void setJobFinished(JobID jobID) {
 		checkNotNull(jobID);
 
-		synchronized (running) {
-			running.remove(jobID);
-			finished.add(jobID);
-		}
-	}
-
-	@Override
-	public boolean isJobRunning(JobID jobID) {
-		checkNotNull(jobID);
-
-		synchronized (running) {
-			return running.contains(jobID);
+		synchronized (jobStatus) {
+			jobStatus.put(jobID, JobSchedulingStatus.DONE);
 		}
 	}
 
 	@Override
 	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) {
 		checkNotNull(jobID);
-
-		synchronized (running) {
-			if (finished.contains(jobID)) {
-				return JobSchedulingStatus.DONE;
-			} else if (running.contains(jobID)) {
-				return JobSchedulingStatus.RUNNING;
-			} else {
-				return JobSchedulingStatus.PENDING;
-			}
+		
+		synchronized (jobStatus) {
+			JobSchedulingStatus status = jobStatus.get(jobID);
+			return status == null ? JobSchedulingStatus.PENDING : status;
 		}
 	}
 
@@ -85,9 +65,8 @@ public class NonHaRegistry implements RunningJobsRegistry {
 	public void clearJob(JobID jobID) {
 		checkNotNull(jobID);
 
-		synchronized (running) {
-			running.remove(jobID);
-			finished.remove(jobID);
+		synchronized (jobStatus) {
+			jobStatus.remove(jobID);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
index 6bebd90..6e02813 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -360,20 +360,22 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
 			// it's okay that job manager wait for the operation complete
 			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
 
-			JobSchedulingStatus schedulingStatus = JobSchedulingStatus.PENDING;
+			final JobSchedulingStatus schedulingStatus;
 			try {
 				schedulingStatus = runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID());
-				if (schedulingStatus.equals(JobSchedulingStatus.DONE)) {
-					log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID());
-					jobFinishedByOther();
-					return;
-				}
-			} catch (Throwable t) {
-				log.error("Could not access status (running/finished) of job {}. ",	jobGraph.getJobID(), t);
+			}
+			catch (Throwable t) {
+				log.error("Could not access status (running/finished) of job {}. ", jobGraph.getJobID(), t);
 				onFatalError(t);
 				return;
 			}
 
+			if (schedulingStatus == JobSchedulingStatus.DONE) {
+				log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID());
+				jobFinishedByOther();
+				return;
+			}
+
 			// Double check the leadership after we confirm that, there is a small chance that multiple
 			// job managers schedule the same job after if they try to recover at the same time.
 			// This will eventually be noticed, but can not be ruled out from the beginning.
@@ -382,7 +384,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
 					// Now set the running status is after getting leader ship and 
 					// set finished status after job in terminated status.
 					// So if finding the job is running, it means someone has already run the job, need recover.
-					if (schedulingStatus.equals(JobSchedulingStatus.PENDING)) {
+					if (schedulingStatus == JobSchedulingStatus.PENDING) {
 						runningJobsRegistry.setJobRunning(jobGraph.getJobID());
 					}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
index 9178684..dd80ada 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -271,7 +271,7 @@ public class MiniClusterJobDispatcher {
 			haServices.getRunningJobsRegistry().clearJob(jobID);
 		}
 		catch (Throwable t) {
-			LOG.warn("Could not clear the job {} at the high-availability services", jobID.toString(), t);
+			LOG.warn("Could not clear job {} at the status registry of the high-availability services", jobID, t);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
index 6c7e249..7ef39de 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.util.ExceptionUtils;
@@ -244,7 +245,8 @@ public class JobLeaderIdService {
 				}
 
 				try {
-					if (runningJobsRegistry.isJobRunning(jobId)) {
+					final JobSchedulingStatus jobStatus = runningJobsRegistry.getJobSchedulingStatus(jobId);
+					if (jobStatus == JobSchedulingStatus.PENDING || jobStatus == JobSchedulingStatus.RUNNING) {
 						if (leaderSessionId == null) {
 							// there is no new leader
 							if (previousJobLeaderId != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
index bbafcf0..b0c7778 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
@@ -30,9 +30,10 @@ import org.junit.rules.TemporaryFolder;
 import java.io.File;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
+/**
+ * Tests for the {@link FsNegativeRunningJobsRegistry} on a local file system.
+ */
 public class FsNegativeRunningJobsRegistryTest extends TestLogger {
 
 	@Rule
@@ -47,23 +48,22 @@ public class FsNegativeRunningJobsRegistryTest extends TestLogger {
 
 		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri));
 
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
+
 		// initially, without any call, the job is pending
-		assertFalse(registry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.PENDING);
+		assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid));
 
 		// after set running, the job is running
 		registry.setJobRunning(jid);
-		assertTrue(registry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.RUNNING);
+		assertEquals(JobSchedulingStatus.RUNNING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.RUNNING, otherRegistry.getJobSchedulingStatus(jid));
 
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
-
-		// another registry should pick this up
-		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
-		assertTrue(otherRegistry.isJobRunning(jid));
-		assertEquals(otherRegistry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid));
 	}
 
 	@Test
@@ -77,22 +77,19 @@ public class FsNegativeRunningJobsRegistryTest extends TestLogger {
 
 		// set the job to finished and validate
 		registry.setJobFinished(jid);
-		assertFalse(registry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
 
 		// set the job to running does not overwrite the finished status
 		registry.setJobRunning(jid);
-		assertTrue(registry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
+		assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid));
 
 		// another registry should pick this up
 		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
-		assertTrue(otherRegistry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.DONE);
+		assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid));
 
 		// clear the running and finished marker, it will be pending
 		otherRegistry.clearJob(jid);
-		assertFalse(otherRegistry.isJobRunning(jid));
-		assertEquals(registry.getJobSchedulingStatus(jid), JobSchedulingStatus.PENDING);
+		assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid));
+		assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0dede9f/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
index 8c91898..b1881e5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
@@ -22,22 +22,21 @@ import org.apache.curator.test.TestingServer;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+
 
 public class ZooKeeperRegistryTest extends TestLogger {
-	private TestingServer testingServer;
 
-	private static Logger LOG = LoggerFactory.getLogger(ZooKeeperRegistryTest.class);
+	private TestingServer testingServer;
 
 	@Before
 	public void before() throws Exception {
@@ -59,29 +58,25 @@ public class ZooKeeperRegistryTest extends TestLogger {
 		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
 		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
-		HighAvailabilityServices zkHaService = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration);
-		RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry();
+		final HighAvailabilityServices zkHaService = new ZookeeperHaServices(
+				ZooKeeperUtils.startCuratorFramework(configuration), Executors.directExecutor(), configuration);
+
+		final RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry();
 
 		try {
 			JobID jobID = JobID.generate();
-			assertFalse(zkRegistry.isJobRunning(jobID));
-			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.PENDING);
+			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
 
 			zkRegistry.setJobRunning(jobID);
-			assertTrue(zkRegistry.isJobRunning(jobID));
-			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.RUNNING);
+			assertEquals(JobSchedulingStatus.RUNNING, zkRegistry.getJobSchedulingStatus(jobID));
 
 			zkRegistry.setJobFinished(jobID);
-			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.DONE);
-			assertFalse(zkRegistry.isJobRunning(jobID));
+			assertEquals(JobSchedulingStatus.DONE, zkRegistry.getJobSchedulingStatus(jobID));
 
 			zkRegistry.clearJob(jobID);
-			assertFalse(zkRegistry.isJobRunning(jobID));
-			assertEquals(zkRegistry.getJobSchedulingStatus(jobID), JobSchedulingStatus.PENDING);
+			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
 		} finally {
-			if (zkHaService != null) {
-				zkHaService.close();
-			}
+			zkHaService.close();
 		}
 	}
 }


[04/11] flink git commit: [FLINK-5929] [tests] Fix SavepointITCase instability

Posted by se...@apache.org.
[FLINK-5929] [tests] Fix SavepointITCase instability

When shutting down the testing cluster it can happen that checkpoint
files lingered around (checkpoints independent of the savepoint).

This commit deactives checkpointing for the test and uses count down
latches to track progress, which also reduces the test time.

This closes #3427


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

Branch: refs/heads/master
Commit: c24c7ec3332d0eb6ebb24eb70c9aabd055cc129f
Parents: 7f244b8
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Feb 28 11:13:28 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../test/checkpointing/SavepointITCase.java     | 200 ++++++-------------
 1 file changed, 62 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c24c7ec3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index ac37009..ee371dd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -24,7 +24,7 @@ import akka.testkit.JavaTestKit;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import java.io.FileNotFoundException;
-import org.apache.commons.io.FileUtils;
+import java.util.concurrent.CountDownLatch;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -54,7 +54,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.state.ChainedStateHandle;
-import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
@@ -95,7 +94,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.runtime.messages.JobManagerMessages.getDisposeSavepointSuccess;
 import static org.junit.Assert.assertEquals;
@@ -116,12 +114,11 @@ public class SavepointITCase extends TestLogger {
 	public TemporaryFolder folder = new TemporaryFolder();
 
 	/**
-	 * Tests that it is possible to submit a job, trigger a savepoint, and
-	 * later restart the job on a new cluster. The savepoint is written to
-	 * a file.
+	 * Triggers a savepoint for a job that uses the FsStateBackend. We expect
+	 * that all checkpoint files are written to a new savepoint directory.
 	 *
 	 * <ol>
-	 * <li>Submit job, wait for some checkpoints to complete</li>
+	 * <li>Submit job, wait for some progress</li>
 	 * <li>Trigger savepoint and verify that savepoint has been created</li>
 	 * <li>Shut down the cluster, re-submit the job from the savepoint,
 	 * verify that the initial state has been reset, and
@@ -131,23 +128,13 @@ public class SavepointITCase extends TestLogger {
 	 * </ol>
 	 */
 	@Test
-	public void testTriggerSavepointAndResume() throws Exception {
+	public void testTriggerSavepointAndResumeWithFileBasedCheckpoints() throws Exception {
 		// Config
-		int numTaskManagers = 2;
-		int numSlotsPerTaskManager = 2;
-		int parallelism = numTaskManagers * numSlotsPerTaskManager;
-
-		// Test deadline
+		final int numTaskManagers = 2;
+		final int numSlotsPerTaskManager = 2;
+		final int parallelism = numTaskManagers * numSlotsPerTaskManager;
 		final Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
-
-		// The number of checkpoints to complete before triggering the savepoint
-		final int numberOfCompletedCheckpoints = 2;
-		final int checkpointingInterval = 100;
-
-		// Temporary directory for file state backend
-		final File tmpDir = folder.newFolder();
-
-		LOG.info("Created temporary directory: " + tmpDir + ".");
+		final File testRoot = folder.newFolder();
 
 		TestingCluster flink = null;
 
@@ -160,70 +147,51 @@ public class SavepointITCase extends TestLogger {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTaskManager);
 
-			final File checkpointDir = new File(tmpDir, "checkpoints");
-			final File savepointRootDir = new File(tmpDir, "savepoints");
+			final File checkpointDir = new File(testRoot, "checkpoints");
+			final File savepointRootDir = new File(testRoot, "savepoints");
 
 			if (!checkpointDir.mkdir() || !savepointRootDir.mkdirs()) {
 				fail("Test setup failed: failed to create temporary directories.");
 			}
 
-			LOG.info("Created temporary checkpoint directory: " + checkpointDir + ".");
-			LOG.info("Created temporary savepoint directory: " + savepointRootDir + ".");
-
+			// Use file based checkpoints
 			config.setString(CoreOptions.STATE_BACKEND, "filesystem");
-			config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY,
-				checkpointDir.toURI().toString());
+			config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, checkpointDir.toURI().toString());
 			config.setString(FsStateBackendFactory.MEMORY_THRESHOLD_CONF_KEY, "0");
-			config.setString(ConfigConstants.SAVEPOINT_DIRECTORY_KEY,
-				savepointRootDir.toURI().toString());
-
-			LOG.info("Flink configuration: " + config + ".");
+			config.setString(ConfigConstants.SAVEPOINT_DIRECTORY_KEY, savepointRootDir.toURI().toString());
 
 			// Start Flink
 			flink = new TestingCluster(config);
-			flink.start();
-
-			// Retrieve the job manager
-			ActorGateway jobManager = Await.result(
-				flink.leaderGateway().future(),
-				deadline.timeLeft());
+			flink.start(true);
 
 			// Submit the job
-			final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000, checkpointingInterval);
+			final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000);
 			final JobID jobId = jobGraph.getJobID();
 
-			// Wait for the source to be notified about the expected number
-			// of completed checkpoints
-			StatefulCounter.resetForTest();
+			// Reset the static test job helpers
+			StatefulCounter.resetForTest(parallelism);
+
+			// Retrieve the job manager
+			ActorGateway jobManager = Await.result(flink.leaderGateway().future(), deadline.timeLeft());
 
 			LOG.info("Submitting job " + jobGraph.getJobID() + " in detached mode.");
 
 			flink.submitJobDetached(jobGraph);
 
-			LOG.info("Waiting for " + numberOfCompletedCheckpoints + " checkpoint complete notifications.");
+			LOG.info("Waiting for some progress.");
 
-			// Wait...
-			StatefulCounter.awaitCompletedCheckpoints(parallelism, numberOfCompletedCheckpoints, deadline.timeLeft().toMillis());
+			StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
-			LOG.info("Received all " + numberOfCompletedCheckpoints +
-				" checkpoint complete notifications.");
-
-			// ...and then trigger the savepoint
 			LOG.info("Triggering a savepoint.");
-
-			Future<Object> savepointPathFuture = jobManager.ask(
-				new TriggerSavepoint(jobId, Option.<String>empty()), deadline.timeLeft());
-
-			final String savepointPath = ((TriggerSavepointSuccess) Await
-				.result(savepointPathFuture, deadline.timeLeft())).savepointPath();
+			Future<Object> savepointPathFuture = jobManager.ask(new TriggerSavepoint(jobId, Option.<String>empty()), deadline.timeLeft());
+			final String savepointPath = ((TriggerSavepointSuccess) Await.result(savepointPathFuture, deadline.timeLeft())).savepointPath();
 			LOG.info("Retrieved savepoint path: " + savepointPath + ".");
 
 			// Retrieve the savepoint from the testing job manager
 			LOG.info("Requesting the savepoint.");
 			Future<Object> savepointFuture = jobManager.ask(new RequestSavepoint(savepointPath), deadline.timeLeft());
 
-			SavepointV1 savepoint = (SavepointV1) ((ResponseSavepoint) Await.result(
-				savepointFuture, deadline.timeLeft())).savepoint();
+			SavepointV1 savepoint = (SavepointV1) ((ResponseSavepoint) Await.result(savepointFuture, deadline.timeLeft())).savepoint();
 			LOG.info("Retrieved savepoint: " + savepointPath + ".");
 
 			// Shut down the Flink cluster (thereby canceling the job)
@@ -243,26 +211,25 @@ public class SavepointITCase extends TestLogger {
 				File savepointDir = files[0];
 				File[] savepointFiles = savepointDir.listFiles();
 				assertNotNull(savepointFiles);
-				assertTrue("Did not write savepoint files to directory",savepointFiles.length > 1);
+
+				// Expect one metadata file and one checkpoint file per stateful
+				// parallel subtask
+				String errMsg = "Did not write expected number of savepoint/checkpoint files to directory: "
+					+ Arrays.toString(savepointFiles);
+				assertEquals(errMsg, 1 + parallelism, savepointFiles.length);
 			} else {
 				fail("Savepoint not created in expected directory");
 			}
 
-			// Only one checkpoint of the savepoint should exist
 			// We currently have the following directory layout: checkpointDir/jobId/chk-ID
 			File jobCheckpoints = new File(checkpointDir, jobId.toString());
 
 			if (jobCheckpoints.exists()) {
 				files = jobCheckpoints.listFiles();
 				assertNotNull("Checkpoint directory empty", files);
-				assertEquals("Checkpoints directory not cleaned up: " + Arrays.toString(files), 0, files.length);
+				assertEquals("Checkpoints directory not clean: " + Arrays.toString(files), 0, files.length);
 			}
 
-			// Only one savepoint should exist
-			files = savepointRootDir.listFiles();
-			assertNotNull("Savepoint directory empty", files);
-			assertEquals("No savepoint found in savepoint directory", 1, files.length);
-
 			// - Verification END ---------------------------------------------
 
 			// Restart the cluster
@@ -274,13 +241,14 @@ public class SavepointITCase extends TestLogger {
 			jobManager = Await.result(flink.leaderGateway().future(), deadline.timeLeft());
 			LOG.info("JobManager: " + jobManager + ".");
 
-			// Reset for restore
-			StatefulCounter.resetForTest();
+			// Reset static test helpers
+			StatefulCounter.resetForTest(parallelism);
 
 			// Gather all task deployment descriptors
 			final Throwable[] error = new Throwable[1];
 			final TestingCluster finalFlink = flink;
 			final Multimap<JobVertexID, TaskDeploymentDescriptor> tdds = HashMultimap.create();
+
 			new JavaTestKit(testActorSystem) {{
 
 				new Within(deadline.timeLeft()) {
@@ -361,10 +329,10 @@ public class SavepointITCase extends TestLogger {
 			}
 
 			// Await state is restored
-			StatefulCounter.awaitStateRestoredFromCheckpoint(deadline.timeLeft().toMillis());
+			StatefulCounter.getRestoreLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 			// Await some progress after restore
-			StatefulCounter.awaitCompletedCheckpoints(parallelism, numberOfCompletedCheckpoints, deadline.timeLeft().toMillis());
+			StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 			// - Verification END ---------------------------------------------
 
@@ -396,7 +364,7 @@ public class SavepointITCase extends TestLogger {
 				}
 			}
 
-			// The checkpoint of the savepoint should have been discarded
+			// The checkpoint files of the savepoint should have been discarded
 			for (File f : checkpointFiles) {
 				errMsg = "Checkpoint file " + f + " not cleaned up properly.";
 				assertFalse(errMsg, f.exists());
@@ -418,10 +386,6 @@ public class SavepointITCase extends TestLogger {
 			if (flink != null) {
 				flink.shutdown();
 			}
-
-			if (tmpDir != null) {
-				FileUtils.deleteDirectory(tmpDir);
-			}
 		}
 	}
 
@@ -467,7 +431,7 @@ public class SavepointITCase extends TestLogger {
 			// Submit the job
 			// Long delay to ensure that the test times out if the job
 			// manager tries to restart the job.
-			final JobGraph jobGraph = createJobGraph(parallelism, numberOfRetries, 3600000, 1000);
+			final JobGraph jobGraph = createJobGraph(parallelism, numberOfRetries, 3600000);
 
 			// Set non-existing savepoint path
 			jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath("unknown path"));
@@ -498,12 +462,10 @@ public class SavepointITCase extends TestLogger {
 	private JobGraph createJobGraph(
 		int parallelism,
 		int numberOfRetries,
-		long restartDelay,
-		int checkpointingInterval) {
+		long restartDelay) {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(parallelism);
-		env.enableCheckpointing(checkpointingInterval);
 		env.disableOperatorChaining();
 		env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(numberOfRetries, restartDelay));
 		env.getConfig().disableSysoutLogging();
@@ -526,7 +488,9 @@ public class SavepointITCase extends TestLogger {
 		@Override
 		public void run(SourceContext<Integer> ctx) throws Exception {
 			while (running) {
-				ctx.collect(1);
+				synchronized (ctx.getCheckpointLock()) {
+					ctx.collect(1);
+				}
 			}
 		}
 
@@ -536,14 +500,12 @@ public class SavepointITCase extends TestLogger {
 		}
 	}
 
-	private static class StatefulCounter
-		extends RichMapFunction<Integer, Integer>
-		implements ListCheckpointed<byte[]>, CheckpointListener {
+	private static class StatefulCounter extends RichMapFunction<Integer, Integer> implements ListCheckpointed<byte[]>{
+
+		private static volatile CountDownLatch progressLatch = new CountDownLatch(0);
+		private static volatile CountDownLatch restoreLatch = new CountDownLatch(0);
 
-		private static final Object checkpointLock = new Object();
-		private static int numCompleteCalls;
-		private static int numRestoreCalls;
-		private static boolean restoredFromCheckpoint;
+		private int numCollectedElements = 0;
 
 		private static final long serialVersionUID = 7317800376639115920L;
 		private byte[] data;
@@ -563,6 +525,11 @@ public class SavepointITCase extends TestLogger {
 			for (int i = 0; i < data.length; i++) {
 				data[i] += 1;
 			}
+
+			if (numCollectedElements++ > 10) {
+				progressLatch.countDown();
+			}
+
 			return value;
 		}
 
@@ -578,65 +545,22 @@ public class SavepointITCase extends TestLogger {
 			}
 			this.data = state.get(0);
 
-			synchronized (checkpointLock) {
-				if (++numRestoreCalls == getRuntimeContext().getNumberOfParallelSubtasks()) {
-					restoredFromCheckpoint = true;
-					checkpointLock.notifyAll();
-				}
-			}
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) throws Exception {
-			synchronized (checkpointLock) {
-				numCompleteCalls++;
-				checkpointLock.notifyAll();
-			}
+			restoreLatch.countDown();
 		}
 
 		// --------------------------------------------------------------------
 
-		static void resetForTest() {
-			synchronized (checkpointLock) {
-				numCompleteCalls = 0;
-				numRestoreCalls = 0;
-				restoredFromCheckpoint = false;
-			}
+		static CountDownLatch getProgressLatch() {
+			return progressLatch;
 		}
 
-		static void awaitCompletedCheckpoints(
-				int parallelism,
-				int expectedNumberOfCompletedCheckpoints,
-				long timeoutMillis) throws InterruptedException, TimeoutException {
-
-			long deadline = System.nanoTime() + timeoutMillis * 1_000_000;
-
-			synchronized (checkpointLock) {
-				// One completion notification per parallel subtask
-				int expectedNumber = parallelism * expectedNumberOfCompletedCheckpoints;
-				while (numCompleteCalls < expectedNumber && System.nanoTime() <= deadline) {
-					checkpointLock.wait();
-				}
-
-				if (numCompleteCalls < expectedNumber) {
-					throw new TimeoutException("Did not complete " + expectedNumberOfCompletedCheckpoints +
-						" within timeout of " + timeoutMillis + " millis.");
-				}
-			}
+		static CountDownLatch getRestoreLatch() {
+			return restoreLatch;
 		}
 
-		static void awaitStateRestoredFromCheckpoint(long timeoutMillis) throws InterruptedException, TimeoutException {
-			long deadline = System.nanoTime() + timeoutMillis * 1_000_000;
-
-			synchronized (checkpointLock) {
-				while (!restoredFromCheckpoint && System.currentTimeMillis() <= deadline) {
-					checkpointLock.wait();
-				}
-
-				if (!restoredFromCheckpoint) {
-					throw new TimeoutException("Did not restore from checkpoint within timeout of " + timeoutMillis + " millis.");
-				}
-			}
+		static void resetForTest(int parallelism) {
+			progressLatch = new CountDownLatch(parallelism);
+			restoreLatch = new CountDownLatch(parallelism);
 		}
 	}
 


[03/11] flink git commit: [FLINK-5133] [core] Add new setResource API for DataStream and DataSet

Posted by se...@apache.org.
[FLINK-5133] [core] Add new setResource API for DataStream and DataSet

This introduces the internals, but does not yet make it public in the API.

This closes #3303


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

Branch: refs/heads/master
Commit: f37ed02909617da4e73a01173b67373369ec3bc8
Parents: c24c7ec
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Authored: Tue Feb 14 12:37:18 2017 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:10 2017 +0100

----------------------------------------------------------------------
 .../flink/api/common/operators/Operator.java    | 36 ++++++++
 .../flink/api/java/operators/DataSink.java      | 64 ++++++++++++++
 .../api/java/operators/DeltaIteration.java      | 65 +++++++++++++-
 .../flink/api/java/operators/Operator.java      | 67 ++++++++++++++
 .../api/java/operators/OperatorTranslation.java | 26 ++++--
 .../flink/api/java/operator/OperatorTest.java   | 15 ++++
 .../apache/flink/optimizer/plan/PlanNode.java   |  9 ++
 .../org/apache/flink/api/scala/DataSet.scala    | 49 ++++++++++-
 .../streaming/api/datastream/DataStream.java    | 19 ++++
 .../api/datastream/DataStreamSink.java          | 36 ++++++++
 .../datastream/SingleOutputStreamOperator.java  | 36 ++++++++
 .../flink/streaming/api/graph/StreamGraph.java  |  7 ++
 .../api/graph/StreamGraphGenerator.java         |  4 +
 .../flink/streaming/api/graph/StreamNode.java   | 16 ++++
 .../transformations/StreamTransformation.java   | 42 +++++++++
 .../flink/streaming/api/DataStreamTest.java     | 92 ++++++++++++++++++++
 .../flink/streaming/api/scala/DataStream.scala  | 33 +++++++
 .../streaming/api/scala/DataStreamTest.scala    | 79 ++++++++++++++++-
 18 files changed, 686 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
index 7e70fd7..a9dedfa 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
@@ -45,6 +45,10 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 		
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;  // the number of parallel instances to use
 
+	private ResourceSpec minResource;			// the minimum resource of the contract instance.
+
+	private ResourceSpec preferredResource;	// the preferred resource of the contract instance.
+
 	/**
 	 * The return type of the user function.
 	 */
@@ -184,6 +188,38 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	public void setParallelism(int parallelism) {
 		this.parallelism = parallelism;
 	}
+
+	/**
+	 * Gets the minimum resource for this contract instance. The minimum resource denotes how many
+	 * resources will be needed in the minimum for the user function during the execution.
+	 *
+	 * @return The minimum resource of this operator.
+	 */
+	public ResourceSpec getMinResource() {
+		return this.minResource;
+	}
+
+	/**
+	 * Gets the preferred resource for this contract instance. The preferred resource denotes how many
+	 * resources will be needed in the maximum for the user function during the execution.
+	 *
+	 * @return The preferred resource of this operator.
+	 */
+	public ResourceSpec getPreferredResource() {
+		return this.preferredResource;
+	}
+
+	/**
+	 * Sets the minimum and preferred resources for this contract instance. The resource denotes
+	 * how many memories and cpu cores of the user function will be consumed during the execution.
+	 *
+	 * @param minResource The minimum resource of this operator.
+	 * @param preferredResource The preferred resource of this operator.
+	 */
+	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+	}
 	
 	
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
index 8b419d9..3be9cc0 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
@@ -29,6 +29,7 @@ import org.apache.flink.api.common.operators.Keys;
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.Order;
 import org.apache.flink.api.common.operators.Ordering;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.typeinfo.NothingTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -51,6 +52,10 @@ public class DataSink<T> {
 	
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
 
+	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+
+	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+
 	private Configuration parameters;
 
 	private int[] sortKeyPositions;
@@ -278,4 +283,63 @@ public class DataSink<T> {
 
 		return this;
 	}
+
+	/**
+	 * Returns the minimum resource of this data sink. If no minimum resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The minimum resource of this data sink.
+	 */
+	public ResourceSpec getMinResource() {
+		return this.minResource;
+	}
+
+	/**
+	 * Returns the preferred resource of this data sink. If no preferred resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The preferred resource of this data sink.
+	 */
+	public ResourceSpec getPreferredResource() {
+		return this.preferredResource;
+	}
+
+	/**
+	 * Sets the minimum and preferred resources for this data sink. This overrides the default empty resource.
+	 *	The minimum resource must be satisfied and the preferred resource specifies the upper bound
+	 * for dynamic resource resize.
+	 *
+	 * @param minResource The minimum resource for this data sink.
+	 * @param preferredResource The preferred resource for this data sink.
+	 * @return The data sink with set minimum and preferred resources.
+	 */
+	/*
+	public DataSink<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		Preconditions.checkNotNull(minResource != null && preferredResource != null,
+				"The min and preferred resources must be not null.");
+		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
+				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+
+		return this;
+	}*/
+
+	/**
+	 * Sets the resource for this data sink. This overrides the default empty minimum and preferred resources.
+	 *
+	 * @param resource The resource for this data sink.
+	 * @return The data sink with set minimum and preferred resources.
+	 */
+	/*
+	public DataSink<T> setResource(ResourceSpec resource) {
+		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
+		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
+
+		this.minResource = resource;
+		this.preferredResource = resource;
+
+		return this;
+	}*/
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
index b97a9de..cf0a63e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.common.aggregators.Aggregator;
 import org.apache.flink.api.common.aggregators.AggregatorRegistry;
 import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
 import org.apache.flink.api.common.operators.Keys;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -62,10 +63,13 @@ public class DeltaIteration<ST, WT> {
 	private String name;
 	
 	private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
+
+	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+
+	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
 	
 	private boolean solutionSetUnManaged;
 
-
 	public DeltaIteration(ExecutionEnvironment context, TypeInformation<ST> type, DataSet<ST> solutionSet, DataSet<WT> workset, Keys<ST> keys, int maxIterations) {
 		initialSolutionSet = solutionSet;
 		initialWorkset = workset;
@@ -192,6 +196,65 @@ public class DeltaIteration<ST, WT> {
 	public int getParallelism() {
 		return parallelism;
 	}
+
+	/**
+	 * Sets the minimum and preferred resources for the iteration. This overrides the default empty resource.
+	 *	The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+	 *
+	 * @param minResource The minimum resource for the iteration.
+	 * @param preferredResource The preferred resource for the iteration.
+	 * @return The iteration with set minimum and preferred resources.
+	 */
+	/*
+	public DeltaIteration<ST, WT> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		Preconditions.checkNotNull(minResource != null && preferredResource != null,
+				"The min and preferred resources must be not null.");
+		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
+				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+
+		return this;
+	}*/
+
+	/**
+	 * Sets the resource for the iteration, and the minimum and preferred resources are the same by default.
+	 *	The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+	 *
+	 * @param resource The resource for the iteration.
+	 * @return The iteration with set minimum and preferred resources.
+	 */
+	/*
+	public DeltaIteration<ST, WT> setResource(ResourceSpec resource) {
+		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
+		Preconditions.checkArgument(resource.isValid(), "The values in resource must be not less than 0.");
+
+		this.minResource = resource;
+		this.preferredResource = resource;
+
+		return this;
+	}*/
+
+	/**
+	 * Gets the minimum resource from this iteration. If no minimum resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The minimum resource of the iteration.
+	 */
+	public ResourceSpec getMinResource() {
+		return this.minResource;
+	}
+
+	/**
+	 * Gets the preferred resource from this iteration. If no preferred resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The preferred resource of the iteration.
+	 */
+	public ResourceSpec getPreferredResource() {
+		return this.preferredResource;
+	}
 	
 	/**
 	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
index 323d23e..79cae14 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.operators;
 
 import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -38,6 +39,10 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	
 	protected int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;
 
+	protected ResourceSpec minResource = ResourceSpec.UNKNOWN;
+
+	protected ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+
 	protected Operator(ExecutionEnvironment context, TypeInformation<OUT> resultType) {
 		super(context, resultType);
 	}
@@ -71,6 +76,26 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	}
 
 	/**
+	 * Returns the minimum resource of this operator. If no minimum resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The minimum resource of this operator.
+	 */
+	public ResourceSpec minResource() {
+		return this.minResource;
+	}
+
+	/**
+	 * Returns the preferred resource of this operator. If no preferred resource has been set,
+	 * it returns the default empty resource.
+	 *
+	 * @return The preferred resource of this operator.
+	 */
+	public ResourceSpec preferredResource() {
+		return this.preferredResource;
+	}
+
+	/**
 	 * Sets the name of this operator. This overrides the default name, which is either
 	 * a generated description of the operation (such as for example "Aggregate(1:SUM, 2:MIN)")
 	 * or the name the user-defined function or input/output format executed by the operator.
@@ -103,4 +128,46 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 		O returnType = (O) this;
 		return returnType;
 	}
+
+	/**
+	 * Sets the minimum and preferred resources for this operator. This overrides the default empty resource.
+	 * The lower and upper resource limits will be considered in dynamic resource resize feature for future plan.
+	 *
+	 * @param minResource The minimum resource for this operator.
+	 * @param preferredResource The preferred resource for this operator.
+	 * @return The operator with set minimum and preferred resources.
+	 */
+	/*
+	public O setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		Preconditions.checkNotNull(minResource != null && preferredResource != null,
+				"The min and preferred resources must be not null.");
+		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
+				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+
+		@SuppressWarnings("unchecked")
+		O returnType = (O) this;
+		return returnType;
+	}*/
+
+	/**
+	 * Sets the resource for this operator. This overrides the default empty minimum and preferred resources.
+	 *
+	 * @param resource The resource for this operator.
+	 * @return The operator with set minimum and preferred resources.
+	 */
+	/*
+	public O setResource(ResourceSpec resource) {
+		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
+		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
+
+		this.minResource = resource;
+		this.preferredResource = resource;
+
+		@SuppressWarnings("unchecked")
+		O returnType = (O) this;
+		return returnType;
+	}*/
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index 88c9c37..909cd32 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -63,7 +63,9 @@ public class OperatorTranslation {
 		
 		// translate the sink itself and connect it to the input
 		GenericDataSinkBase<T> translatedSink = sink.translateToDataFlow(input);
-				
+
+		translatedSink.setResource(sink.getMinResource(), sink.getPreferredResource());
+
 		return translatedSink;
 	}
 	
@@ -91,19 +93,31 @@ public class OperatorTranslation {
 		Operator<T> dataFlowOp;
 		
 		if (dataSet instanceof DataSource) {
-			dataFlowOp = ((DataSource<T>) dataSet).translateToDataFlow();
+			DataSource<T> dataSource = (DataSource<T>) dataSet;
+			dataFlowOp = dataSource.translateToDataFlow();
+			dataFlowOp.setResource(dataSource.minResource(), dataSource.preferredResource());
 		}
 		else if (dataSet instanceof SingleInputOperator) {
-			dataFlowOp = translateSingleInputOperator((SingleInputOperator<?, ?, ?>) dataSet);
+			SingleInputOperator<?, ?, ?> singleInputOperator = (SingleInputOperator<?, ?, ?>) dataSet;
+			dataFlowOp = translateSingleInputOperator(singleInputOperator);
+			dataFlowOp.setResource(singleInputOperator.minResource, singleInputOperator.preferredResource());
 		}
 		else if (dataSet instanceof TwoInputOperator) {
-			dataFlowOp = translateTwoInputOperator((TwoInputOperator<?, ?, ?, ?>) dataSet);
+			TwoInputOperator<?, ?, ?, ?> twoInputOperator = (TwoInputOperator<?, ?, ?, ?>) dataSet;
+			dataFlowOp = translateTwoInputOperator(twoInputOperator);
+			dataFlowOp.setResource(twoInputOperator.minResource(), twoInputOperator.preferredResource());
 		}
 		else if (dataSet instanceof BulkIterationResultSet) {
-			dataFlowOp = translateBulkIteration((BulkIterationResultSet<?>) dataSet);
+			BulkIterationResultSet bulkIterationResultSet = (BulkIterationResultSet<?>) dataSet;
+			dataFlowOp = translateBulkIteration(bulkIterationResultSet);
+			dataFlowOp.setResource(bulkIterationResultSet.getIterationHead().minResource(),
+					bulkIterationResultSet.getIterationHead().preferredResource());
 		}
 		else if (dataSet instanceof DeltaIterationResultSet) {
-			dataFlowOp = translateDeltaIteration((DeltaIterationResultSet<?, ?>) dataSet);
+			DeltaIterationResultSet deltaIterationResultSet = (DeltaIterationResultSet<?, ?>) dataSet;
+			dataFlowOp = translateDeltaIteration(deltaIterationResultSet);
+			dataFlowOp.setResource(deltaIterationResultSet.getIterationHead().getMinResource(),
+					deltaIterationResultSet.getIterationHead().getPreferredResource());
 		}
 		else if (dataSet instanceof DeltaIteration.SolutionSetPlaceHolder || dataSet instanceof DeltaIteration.WorksetPlaceHolder) {
 			throw new InvalidProgramException("A data set that is part of a delta iteration was used as a sink or action."

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
index a69ca3c..992acc9 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.api.java.operator;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.Operator;
 import org.apache.flink.api.java.typeutils.ValueTypeInfo;
@@ -45,6 +46,20 @@ public class OperatorTest {
 		assertEquals(parallelism, operator.getParallelism());
 	}
 
+	/*
+	@Test
+	public void testConfigurationOfResource() {
+		Operator operator = new MockOperator();
+
+		// verify explicit change in resource
+		ResourceSpec minResource = new ResourceSpec(1.0, 100, 0, 0, 0);
+		ResourceSpec preferredResource = new ResourceSpec(2.0, 200, 0, 0, 0);
+		operator.setResource(minResource, preferredResource);
+
+		assertEquals(minResource, operator.getMinResource());
+		assertEquals(preferredResource, operator.getPreferredResource());
+	}*/
+
 	private class MockOperator extends Operator {
 		public MockOperator() {
 			super(ExecutionEnvironment.createCollectionsEnvironment(), ValueTypeInfo.NULL_VALUE_TYPE_INFO);

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
index b30fa36..4ef91b3 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
@@ -19,6 +19,7 @@
 package org.apache.flink.optimizer.plan;
 
 import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.operators.util.FieldSet;
 import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.optimizer.costs.Costs;
@@ -308,6 +309,14 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 	public int getParallelism() {
 		return this.parallelism;
 	}
+
+	public ResourceSpec getMinResource() {
+		return this.template.getOperator().getMinResource();
+	}
+
+	public ResourceSpec getPreferredResource() {
+		return this.template.getOperator().getPreferredResource();
+	}
 	
 	public long getGuaranteedAvailableMemory() {
 		return this.template.getMinimalMemoryAcrossAllSubTasks();

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 4e7be04..5cfb601 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.accumulators.SerializedListAccumulator
 import org.apache.flink.api.common.aggregators.Aggregator
 import org.apache.flink.api.common.functions._
 import org.apache.flink.api.common.io.{FileOutputFormat, OutputFormat}
-import org.apache.flink.api.common.operators.{Keys, Order}
+import org.apache.flink.api.common.operators.{ResourceSpec, Keys, Order}
 import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
 import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
 import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod
@@ -178,6 +178,53 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
   }
 
   /**
+   * Sets the minimum and preferred resources of this operation.
+   */
+  /*
+  def resource(minResource: ResourceSpec, preferredResource: ResourceSpec) : Unit = {
+    javaSet match {
+      case ds: DataSource[_] => ds.setResource(minResource, preferredResource)
+      case op: Operator[_, _] => op.setResource(minResource, preferredResource)
+      case di: DeltaIterationResultSet[_, _] =>
+        di.getIterationHead.setResource(minResource, preferredResource)
+      case _ =>
+        throw new UnsupportedOperationException("Operator does not support " +
+          "configuring custom resources specs.")
+    }
+    this
+  }*/
+
+  /**
+   * Sets the resource of this operation.
+   */
+  /*
+  def resource(resource: ResourceSpec) : Unit = {
+    this.resource(resource, resource)
+  }*/
+
+  /**
+   * Returns the minimum resource of this operation.
+   */
+  def minResource: ResourceSpec = javaSet match {
+    case ds: DataSource[_] => ds.minResource()
+    case op: Operator[_, _] => op.minResource
+    case _ =>
+      throw new UnsupportedOperationException("Operator does not support " +
+        "configuring custom resources specs.")
+  }
+
+  /**
+   * Returns the preferred resource of this operation.
+   */
+  def preferredResource: ResourceSpec = javaSet match {
+    case ds: DataSource[_] => ds.preferredResource()
+    case op: Operator[_, _] => op.preferredResource
+    case _ =>
+      throw new UnsupportedOperationException("Operator does not support " +
+        "configuring custom resources specs.")
+  }
+
+  /**
    * Registers an [[org.apache.flink.api.common.aggregators.Aggregator]]
    * for the iteration. Aggregators can be used to maintain simple statistics during the
    * iteration, such as number of elements processed. The aggregators compute global aggregates:

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 204557d..ae1c39a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -32,6 +32,7 @@ import org.apache.flink.api.common.functions.RichFilterFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
 import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -143,6 +144,24 @@ public class DataStream<T> {
 	}
 
 	/**
+	 * Gets the minimum resource for this operator.
+	 *
+	 * @return The minimum resource set for this operator.
+	 */
+	public ResourceSpec minResource() {
+		return transformation.getMinResource();
+	}
+
+	/**
+	 * Gets the preferred resource for this operator.
+	 *
+	 * @return The preferred resource set for this operator.
+	 */
+	public ResourceSpec preferredResource() {
+		return transformation.getPreferredResource();
+	}
+
+	/**
 	 * Gets the type of the stream.
 	 *
 	 * @return The type of the datastream.

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index 0c9378b..69e21d6 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -114,6 +114,42 @@ public class DataStreamSink<T> {
 	}
 
 	/**
+	 * Sets the minimum and preferred resources for this sink, and the lower and upper resource limits will
+	 * be considered in resource resize feature for future plan.
+	 *
+	 * @param minResource The minimum resource for this sink.
+	 * @param preferredResource The preferred resource for this sink
+	 * @return The sink with set minimum and preferred resources.
+	 */
+	/*
+	public DataStreamSink<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		Preconditions.checkNotNull(minResource != null && preferredResource != null,
+				"The min and preferred resources must be not null.");
+		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
+				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+
+		transformation.setResource(minResource, preferredResource);
+
+		return this;
+	}*/
+
+	/**
+	 * Sets the resource for this sink, the minimum and preferred resources are the same by default.
+	 *
+	 * @param resource The resource for this sink.
+	 * @return The sink with set minimum and preferred resources.
+	 */
+	/*
+	public DataStreamSink<T> setResource(ResourceSpec resource) {
+		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
+		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
+
+		transformation.setResource(resource, resource);
+
+		return this;
+	}*/
+
+	/**
 	 * Turns off chaining for this operator so thread co-location will not be
 	 * used as an optimization.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 9dd60b7..d856603 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -154,6 +154,42 @@ public class SingleOutputStreamOperator<T> extends DataStream<T> {
 		return this;
 	}
 
+	/**
+	 * Sets the minimum and preferred resources for this operator, and the lower and upper resource limits will
+	 * be considered in dynamic resource resize feature for future plan.
+	 *
+	 * @param minResource The minimum resource for this operator.
+	 * @param preferredResource The preferred resource for this operator.
+	 * @return The operator with set minimum and preferred resources.
+	 */
+	/*
+	public SingleOutputStreamOperator<T> setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		Preconditions.checkArgument(minResource != null && preferredResource != null,
+				"The min and preferred resources must be not null.");
+		Preconditions.checkArgument(minResource.isValid() && preferredResource.isValid() && minResource.lessThanOrEqual(preferredResource),
+				"The values in resource must be not less than 0 and the preferred resource must be greater than the min resource.");
+
+		transformation.setResource(minResource, preferredResource);
+
+		return this;
+	}*/
+
+	/**
+	 * Sets the resource for this operator, the minimum and preferred resources are the same by default.
+	 *
+	 * @param resource The resource for this operator.
+	 * @return The operator with set minimum and preferred resources.
+	 */
+	/*
+	public SingleOutputStreamOperator<T> setResource(ResourceSpec resource) {
+		Preconditions.checkNotNull(resource != null, "The resource must be not null.");
+		Preconditions.checkArgument(resource.isValid(), "The resource values must be greater than 0.");
+
+		transformation.setResource(resource, resource);
+
+		return this;
+	}*/
+
 	private boolean canBeParallel() {
 		return !nonParallel;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index 696c04b..fcbc607 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -32,6 +32,7 @@ import java.util.Set;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -413,6 +414,12 @@ public class StreamGraph extends StreamingPlan {
 		}
 	}
 
+	public void setResource(int vertexID, ResourceSpec minResource, ResourceSpec preferredResource) {
+		if (getStreamNode(vertexID) != null) {
+			getStreamNode(vertexID).setResource(minResource, preferredResource);
+		}
+	}
+
 	public void setOneInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector, TypeSerializer<?> keySerializer) {
 		StreamNode node = getStreamNode(vertexID);
 		node.setStatePartitioner1(keySelector);

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
index ddd0515..af92421 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -202,6 +202,10 @@ public class StreamGraphGenerator {
 			streamGraph.setTransformationUserHash(transform.getId(), transform.getUserProvidedNodeHash());
 		}
 
+		if (transform.getMinResource() != null && transform.getPreferredResource() != null) {
+			streamGraph.setResource(transform.getId(), transform.getMinResource(), transform.getPreferredResource());
+		}
+
 		return transformedIds;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
index 19a3699..0bf9adf 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.graph;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
@@ -48,6 +49,8 @@ public class StreamNode implements Serializable {
 	 * dynamic scaling and the number of key groups used for partitioned state.
 	 */
 	private int maxParallelism;
+	private ResourceSpec minResource;
+	private ResourceSpec preferredResource;
 	private Long bufferTimeout = null;
 	private final String operatorName;
 	private String slotSharingGroup;
@@ -165,6 +168,19 @@ public class StreamNode implements Serializable {
 		this.maxParallelism = maxParallelism;
 	}
 
+	public ResourceSpec getMinResource() {
+		return minResource;
+	}
+
+	public ResourceSpec getPreferredResource() {
+		return preferredResource;
+	}
+
+	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+	}
+
 	public Long getBufferTimeout() {
 		return bufferTimeout != null ? bufferTimeout : env.getBufferTimeout();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
index 5e1b3e2..1d22454 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.transformations;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.MissingTypeInfo;
 import org.apache.flink.streaming.api.graph.StreamGraph;
@@ -126,6 +127,18 @@ public abstract class StreamTransformation<T> {
 	private int maxParallelism = -1;
 
 	/**
+	 *  The minimum resource for this stream transformation. It defines the lower limit for
+	 *  dynamic resource resize in future plan.
+	 */
+	private ResourceSpec minResource = ResourceSpec.UNKNOWN;
+
+	/**
+	 *  The preferred resource for this stream transformation. It defines the upper limit for
+	 *  dynamic resource resize in future plan.
+	 */
+	private ResourceSpec preferredResource = ResourceSpec.UNKNOWN;
+
+	/**
 	 * User-specified ID for this transformation. This is used to assign the
 	 * same operator ID across job restarts. There is also the automatically
 	 * generated {@link #id}, which is assigned from a static counter. That
@@ -214,6 +227,35 @@ public abstract class StreamTransformation<T> {
 	}
 
 	/**
+	 * Sets the minimum and preferred resources for this stream transformation.
+	 *
+	 * @param minResource The minimum resource of this transformation.
+	 * @param preferredResource The preferred resource of this transformation.
+	 */
+	public void setResource(ResourceSpec minResource, ResourceSpec preferredResource) {
+		this.minResource = minResource;
+		this.preferredResource = preferredResource;
+	}
+
+	/**
+	 * Gets the minimum resource of this stream transformation.
+	 *
+	 * @return The minimum resource of this transformation.
+	 */
+	public ResourceSpec getMinResource() {
+		return minResource;
+	}
+
+	/**
+	 * Gets the preferred resource of this stream transformation.
+	 *
+	 * @return The preferred resource of this transformation.
+	 */
+	public ResourceSpec getPreferredResource() {
+		return preferredResource;
+	}
+
+	/**
 	 * Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID.
 	 * <p/>
 	 * <p>The user provided hash is an alternative to the generated hashes, that is considered when identifying an

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index eaac6b8..12af1d4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -501,6 +502,97 @@ public class DataStreamTest {
 		assertEquals(4, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
 	}
 
+	/**
+	 * Tests whether resource gets set.
+	 */
+	/*
+	@Test
+	public void testResource() {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		ResourceSpec minResource1 = new ResourceSpec(1.0, 100);
+		ResourceSpec preferredResource1 = new ResourceSpec(2.0, 200);
+
+		ResourceSpec minResource2 = new ResourceSpec(1.0, 200);
+		ResourceSpec preferredResource2 = new ResourceSpec(2.0, 300);
+
+		ResourceSpec minResource3 = new ResourceSpec(1.0, 300);
+		ResourceSpec preferredResource3 = new ResourceSpec(2.0, 400);
+
+		ResourceSpec minResource4 = new ResourceSpec(1.0, 400);
+		ResourceSpec preferredResource4 = new ResourceSpec(2.0, 500);
+
+		ResourceSpec minResource5 = new ResourceSpec(1.0, 500);
+		ResourceSpec preferredResource5 = new ResourceSpec(2.0, 600);
+
+		ResourceSpec minResource6 = new ResourceSpec(1.0, 600);
+		ResourceSpec preferredResource6 = new ResourceSpec(2.0, 700);
+
+		ResourceSpec minResource7 = new ResourceSpec(1.0, 700);
+		ResourceSpec maxResource7 = new ResourceSpec(2.0, 800);
+
+		DataStream<Long> source1 = env.generateSequence(0, 0).setResource(minResource1, preferredResource1);
+		DataStream<Long> map1 = source1.map(new MapFunction<Long, Long>() {
+			@Override
+			public Long map(Long value) throws Exception {
+				return null;
+			}
+		}).setResource(minResource2, preferredResource2);
+
+		DataStream<Long> source2 = env.generateSequence(0, 0).setResource(minResource3, preferredResource3);
+		DataStream<Long> map2 = source2.map(new MapFunction<Long, Long>() {
+			@Override
+			public Long map(Long value) throws Exception {
+				return null;
+			}
+		}).setResource(minResource4, preferredResource4);
+
+		DataStream<Long> connected = map1.connect(map2)
+				.flatMap(new CoFlatMapFunction<Long, Long, Long>() {
+					@Override
+					public void flatMap1(Long value, Collector<Long> out) throws Exception {
+					}
+					@Override
+					public void flatMap2(Long value, Collector<Long> out) throws Exception {
+					}
+				}).setResource(minResource5, preferredResource5);
+
+		DataStream<Long> windowed = connected
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(10)))
+				.fold(0L, new FoldFunction<Long, Long>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Long fold(Long accumulator, Long value) throws Exception {
+						return null;
+					}
+				}).setResource(minResource6, preferredResource6);
+
+		DataStreamSink<Long> sink = windowed.print().setResource(minResource7, maxResource7);
+
+		assertEquals(minResource1, env.getStreamGraph().getStreamNode(source1.getId()).getMinResource());
+		assertEquals(preferredResource1, env.getStreamGraph().getStreamNode(source1.getId()).getPreferredResource());
+
+		assertEquals(minResource2, env.getStreamGraph().getStreamNode(map1.getId()).getMinResource());
+		assertEquals(preferredResource2, env.getStreamGraph().getStreamNode(map1.getId()).getPreferredResource());
+
+		assertEquals(minResource3, env.getStreamGraph().getStreamNode(source2.getId()).getMinResource());
+		assertEquals(preferredResource3, env.getStreamGraph().getStreamNode(source2.getId()).getPreferredResource());
+
+		assertEquals(minResource4, env.getStreamGraph().getStreamNode(map2.getId()).getMinResource());
+		assertEquals(preferredResource4, env.getStreamGraph().getStreamNode(map2.getId()).getPreferredResource());
+
+		assertEquals(minResource5, env.getStreamGraph().getStreamNode(connected.getId()).getMinResource());
+		assertEquals(preferredResource5, env.getStreamGraph().getStreamNode(connected.getId()).getPreferredResource());
+
+		assertEquals(minResource6, env.getStreamGraph().getStreamNode(windowed.getId()).getMinResource());
+		assertEquals(preferredResource6, env.getStreamGraph().getStreamNode(windowed.getId()).getPreferredResource());
+
+		assertEquals(minResource7, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getMinResource());
+		assertEquals(maxResource7, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getPreferredResource());
+	}*/
+
 	@Test
 	public void testTypeInfo() {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index ba92f86..e42fb3f 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -22,6 +22,7 @@ import org.apache.flink.annotation.{Internal, Public, PublicEvolving}
 import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner}
 import org.apache.flink.api.common.io.OutputFormat
+import org.apache.flink.api.common.operators.ResourceSpec
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.java.tuple.{Tuple => JavaTuple}
@@ -145,6 +146,38 @@ class DataStream[T](stream: JavaStream[T]) {
   }
 
   /**
+   * Returns the minimum resource of this operation.
+   */
+  def minResource: ResourceSpec = stream.minResource()
+
+  /**
+   * Returns the preferred resource of this operation.
+   */
+  def preferredResource: ResourceSpec = stream.preferredResource()
+
+  /**
+   * Sets the minimum and preferred resources of this operation.
+   */
+  /*
+  def resource(minResource: ResourceSpec, preferredResource: ResourceSpec) : DataStream[T] =
+    stream match {
+      case stream : SingleOutputStreamOperator[T] => asScalaStream(stream.setResource(
+        minResource, preferredResource))
+      case _ =>
+        throw new UnsupportedOperationException("Operator does not support " +
+          "configuring custom resources specs.")
+      this
+  }*/
+
+  /**
+   * Sets the resource of this operation.
+   */
+  /*
+  def resource(resource: ResourceSpec) : Unit = {
+    this.resource(resource, resource)
+  }*/
+
+  /**
    * Gets the name of the current data stream. This name is
    * used by the visualization and logging during runtime.
    *

http://git-wip-us.apache.org/repos/asf/flink/blob/f37ed029/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index adb59f2..841567a 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.scala
 import java.lang
 
 import org.apache.flink.api.common.functions._
+import org.apache.flink.api.common.operators.ResourceSpec
 import org.apache.flink.api.java.typeutils.TypeExtractor
 import org.apache.flink.streaming.api.collector.selector.OutputSelector
 import org.apache.flink.streaming.api.functions.ProcessFunction
@@ -34,7 +35,7 @@ import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
 import org.apache.flink.streaming.runtime.partitioner._
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.util.Collector
-import org.junit.Assert.fail
+import org.junit.Assert._
 import org.junit.Test
 
 class DataStreamTest extends StreamingMultipleProgramsTestBase {
@@ -291,6 +292,82 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert(4 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
   }
 
+  /**
+   * Tests whether resource gets set.
+   */
+  /*
+  @Test
+  def testResource() {
+    val env: StreamExecutionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val minResource1: ResourceSpec = new ResourceSpec(1.0, 100)
+    val preferredResource1: ResourceSpec = new ResourceSpec(2.0, 200)
+    val minResource2: ResourceSpec = new ResourceSpec(1.0, 200)
+    val preferredResource2: ResourceSpec = new ResourceSpec(2.0, 300)
+    val minResource3: ResourceSpec = new ResourceSpec(1.0, 300)
+    val preferredResource3: ResourceSpec = new ResourceSpec(2.0, 400)
+    val minResource4: ResourceSpec = new ResourceSpec(1.0, 400)
+    val preferredResource4: ResourceSpec = new ResourceSpec(2.0, 500)
+    val minResource5: ResourceSpec = new ResourceSpec(1.0, 500)
+    val preferredResource5: ResourceSpec = new ResourceSpec(2.0, 600)
+    val minResource6: ResourceSpec = new ResourceSpec(1.0, 600)
+    val preferredResource6: ResourceSpec = new ResourceSpec(2.0, 700)
+    val minResource7: ResourceSpec = new ResourceSpec(1.0, 700)
+    val preferredResource7: ResourceSpec = new ResourceSpec(2.0, 800)
+
+    val source1: DataStream[Long] = env.generateSequence(0, 0)
+      .resource(minResource1, preferredResource1)
+    val map1: DataStream[String] = source1.map(x => "")
+      .resource(minResource2, preferredResource2)
+    val source2: DataStream[Long] = env.generateSequence(0, 0)
+      .resource(minResource3, preferredResource3)
+    val map2: DataStream[String] = source2.map(x => "")
+      .resource(minResource4, preferredResource4)
+
+    val connected: DataStream[String] = map1.connect(map2)
+      .flatMap({ (in, out: Collector[(String)]) => }, { (in, out: Collector[(String)]) => })
+      .resource(minResource5, preferredResource5)
+
+    val windowed  = connected
+      .windowAll(GlobalWindows.create())
+      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
+      .fold("")((accumulator: String, value: String) => "")
+      .resource(minResource6, preferredResource6)
+
+    var sink = windowed.print().resource(minResource7, preferredResource7)
+
+    val plan = env.getExecutionPlan
+
+    assertEquals(minResource1, env.getStreamGraph.getStreamNode(source1.getId).
+      getMinResource)
+    assertEquals(preferredResource1, env.getStreamGraph.getStreamNode(source1.getId).
+      getPreferredResource)
+    assertEquals(minResource2, env.getStreamGraph.getStreamNode(map1.getId).
+      getMinResource)
+    assertEquals(preferredResource2, env.getStreamGraph.getStreamNode(map1.getId).
+      getPreferredResource)
+    assertEquals(minResource3, env.getStreamGraph.getStreamNode(source2.getId).
+      getMinResource)
+    assertEquals(preferredResource3, env.getStreamGraph.getStreamNode(source2.getId).
+      getPreferredResource)
+    assertEquals(minResource4, env.getStreamGraph.getStreamNode(map2.getId).
+      getMinResource)
+    assertEquals(preferredResource4, env.getStreamGraph.getStreamNode(map2.getId).
+      getPreferredResource)
+    assertEquals(minResource5, env.getStreamGraph.getStreamNode(connected.getId).
+      getMinResource)
+    assertEquals(preferredResource5, env.getStreamGraph.getStreamNode(connected.getId).
+      getPreferredResource)
+    assertEquals(minResource6, env.getStreamGraph.getStreamNode(windowed.getId).
+      getMinResource)
+    assertEquals(preferredResource6, env.getStreamGraph.getStreamNode(windowed.getId).
+      getPreferredResource)
+    assertEquals(minResource7, env.getStreamGraph.getStreamNode(
+      sink.getPreferredResource.getId).getMinResource)
+    assertEquals(preferredResource7, env.getStreamGraph.getStreamNode(
+      sink.getPreferredResource.getId).getPreferredResource)
+  }*/
+
   @Test
   def testTypeInfo() {
     val env = StreamExecutionEnvironment.getExecutionEnvironment


[09/11] flink git commit: [FLINK-5928] [checkpoints] Add CheckpointCoordinatorExternalizedCheckpointsTest

Posted by se...@apache.org.
[FLINK-5928] [checkpoints] Add CheckpointCoordinatorExternalizedCheckpointsTest

Problem: there were only unit tests for the checkpoint instances available
that don't test the behaviour of the checkpoint coordinator with respect
to externalized checkpoints.

This closes #3424


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

Branch: refs/heads/master
Commit: c477d87c68f2da4340c8d469e1b4331e6a660ef0
Parents: 3446e66
Author: Ufuk Celebi <uc...@apache.org>
Authored: Mon Feb 27 16:12:37 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 19:02:13 2017 +0100

----------------------------------------------------------------------
 .../runtime/checkpoint/PendingCheckpoint.java   |  24 ++-
 .../checkpoint/savepoint/SavepointStore.java    |  47 ++++-
 ...tCoordinatorExternalizedCheckpointsTest.java | 197 +++++++++++++++++++
 .../checkpoint/CheckpointCoordinatorTest.java   |   2 +-
 .../savepoint/SavepointStoreTest.java           |  23 ++-
 5 files changed, 282 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c477d87c/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index 2c392b8..6c9dbaf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -210,6 +210,7 @@ public class PendingCheckpoint {
 	}
 
 	public CompletedCheckpoint finalizeCheckpointExternalized() throws IOException {
+
 		synchronized (lock) {
 			checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet.");
 
@@ -222,10 +223,27 @@ public class PendingCheckpoint {
 				//            but the checkpoints think more generic. we need to work with file handles
 				//            here until the savepoint serializer accepts a generic stream factory
 
-				final FileStateHandle metadataHandle = SavepointStore.storeSavepointToHandle(targetDirectory, savepoint);
-				final String externalPointer = metadataHandle.getFilePath().getParent().toString();
+				// We have this branch here, because savepoints and externalized checkpoints
+				// currently behave differently.
+				// Savepoints:
+				//   - Metadata file in unique directory
+				//   - External pointer can be the directory
+				// Externalized checkpoints:
+				//   - Multiple metadata files per directory possible (need to be unique)
+				//   - External pointer needs to be the file itself
+				//
+				// This should be unified as part of the JobManager metadata stream factories.
+				if (props.isSavepoint()) {
+					final FileStateHandle metadataHandle = SavepointStore.storeSavepointToHandle(targetDirectory, savepoint);
+					final String externalPointer = metadataHandle.getFilePath().getParent().toString();
+	
+					return finalizeInternal(metadataHandle, externalPointer);
+				} else {
+					final FileStateHandle metadataHandle = SavepointStore.storeExternalizedCheckpointToHandle(targetDirectory, savepoint);
+					final String externalPointer = metadataHandle.getFilePath().toString();
 
-				return finalizeInternal(metadataHandle, externalPointer);
+					return finalizeInternal(metadataHandle, externalPointer);
+				}
 			}
 			catch (Throwable t) {
 				onCompletionPromise.completeExceptionally(t);

http://git-wip-us.apache.org/repos/asf/flink/blob/c477d87c/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
index 5c8ac6b..7beb1b8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
@@ -60,7 +60,13 @@ public class SavepointStore {
 	/** Magic number for sanity checks against stored savepoints. */
 	public static final int MAGIC_NUMBER = 0x4960672d;
 
-	private static final String META_DATA_FILE = "_metadata ";
+	private static final String SAVEPOINT_METADATA_FILE = "_metadata";
+
+	/**
+	 * Metadata file for an externalized checkpoint, random suffix added
+	 * during store, because the parent directory is not unique.
+	 */
+	static final String EXTERNALIZED_CHECKPOINT_METADATA_FILE = "checkpoint_metadata-";
 
 	/**
 	 * Creates a savepoint directory.
@@ -122,7 +128,8 @@ public class SavepointStore {
 	 */
 	public static <T extends Savepoint> String storeSavepoint(String directory, T savepoint) throws IOException {
 		// write and create the file handle
-		FileStateHandle metadataFileHandle = storeSavepointToHandle(directory, savepoint);
+		FileStateHandle metadataFileHandle = storeSavepointToHandle(directory,
+			SAVEPOINT_METADATA_FILE, savepoint);
 
 		// we return the savepoint directory path here!
 		// The directory path also works to resume from and is more elegant than the direct
@@ -135,19 +142,47 @@ public class SavepointStore {
 	 *
 	 * @param directory Target directory to store savepoint in
 	 * @param savepoint Savepoint to be stored
-	 *                     
+	 *
+	 * @return State handle to the checkpoint metadata
+	 * @throws IOException Failures during store are forwarded
+	 */
+	public static <T extends Savepoint> FileStateHandle storeSavepointToHandle(String directory, T savepoint) throws IOException {
+		return storeSavepointToHandle(directory, SAVEPOINT_METADATA_FILE, savepoint);
+	}
+
+	/**
+	 * Stores the externalized checkpoint metadata file to a state handle.
+	 *
+	 * @param directory Target directory to store savepoint in
+	 * @param savepoint Savepoint to be stored
+	 *
+	 * @return State handle to the checkpoint metadata
+	 * @throws IOException Failures during store are forwarded
+	 */
+	public static <T extends Savepoint> FileStateHandle storeExternalizedCheckpointToHandle(String directory, T savepoint) throws IOException {
+		String fileName = FileUtils.getRandomFilename(EXTERNALIZED_CHECKPOINT_METADATA_FILE);
+		return storeSavepointToHandle(directory, fileName, savepoint);
+	}
+
+	/**
+	 * Stores the savepoint metadata file to a state handle.
+	 *
+	 * @param directory Target directory to store savepoint in
+	 * @param savepoint Savepoint to be stored
+	 *
 	 * @return State handle to the checkpoint metadata
 	 * @throws IOException Failures during store are forwarded
 	 */
-	public static <T extends Savepoint> FileStateHandle storeSavepointToHandle(
+	static <T extends Savepoint> FileStateHandle storeSavepointToHandle(
 			String directory,
+			String filename,
 			T savepoint) throws IOException {
 
 		checkNotNull(directory, "Target directory");
 		checkNotNull(savepoint, "Savepoint");
 
 		final Path basePath = new Path(directory);
-		final Path metadataFilePath = new Path(basePath, META_DATA_FILE);
+		final Path metadataFilePath = new Path(basePath, filename);
 
 		final FileSystem fs = FileSystem.get(basePath.toUri());
 
@@ -219,7 +254,7 @@ public class SavepointStore {
 
 		// If this is a directory, we need to find the meta data file
 		if (status.isDir()) {
-			Path candidatePath = new Path(path, META_DATA_FILE);
+			Path candidatePath = new Path(path, SAVEPOINT_METADATA_FILE);
 			if (fs.exists(candidatePath)) {
 				path = candidatePath;
 				LOG.info("Using savepoint file in {}", path);

http://git-wip-us.apache.org/repos/asf/flink/blob/c477d87c/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java
new file mode 100644
index 0000000..9f94f2f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.checkpoint;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointLoader;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * CheckpointCoordinator tests for externalized checkpoints.
+ *
+ * <p>This is separate from {@link CheckpointCoordinatorTest}, because that
+ * test is already huge and covers many different configurations.
+ */
+public class CheckpointCoordinatorExternalizedCheckpointsTest {
+
+	@Rule
+	public TemporaryFolder tmp = new TemporaryFolder();
+
+	/**
+	 * Triggers multiple externalized checkpoints and verifies that the metadata
+	 * files have been created.
+	 */
+	@Test
+	public void testTriggerAndConfirmSimpleExternalizedCheckpoint()
+		throws Exception {
+		final JobID jid = new JobID();
+
+		final ExternalizedCheckpointSettings externalizedCheckpointSettings =
+			ExternalizedCheckpointSettings.externalizeCheckpoints(false);
+
+		final File checkpointDir = tmp.newFolder();
+
+		// create some mock Execution vertices that receive the checkpoint trigger messages
+		final ExecutionAttemptID attemptID1 = new ExecutionAttemptID();
+		final ExecutionAttemptID attemptID2 = new ExecutionAttemptID();
+		ExecutionVertex vertex1 = CheckpointCoordinatorTest.mockExecutionVertex(attemptID1);
+		ExecutionVertex vertex2 = CheckpointCoordinatorTest.mockExecutionVertex(attemptID2);
+
+		Map<JobVertexID, ExecutionJobVertex> jobVertices = new HashMap<>();
+		jobVertices.put(vertex1.getJobvertexId(), vertex1.getJobVertex());
+		jobVertices.put(vertex2.getJobvertexId(), vertex2.getJobVertex());
+
+		// set up the coordinator and validate the initial state
+		CheckpointCoordinator coord = new CheckpointCoordinator(
+			jid,
+			600000,
+			600000,
+			0,
+			Integer.MAX_VALUE,
+			externalizedCheckpointSettings,
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new StandaloneCheckpointIDCounter(),
+			new StandaloneCompletedCheckpointStore(1),
+			checkpointDir.getAbsolutePath(),
+			Executors.directExecutor());
+
+		assertEquals(0, coord.getNumberOfPendingCheckpoints());
+		assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
+
+		// ---------------
+		// trigger checkpoint 1
+		// ---------------
+
+		{
+			final long timestamp1 = System.currentTimeMillis();
+
+			coord.triggerCheckpoint(timestamp1, false);
+
+			long checkpointId1 = coord.getPendingCheckpoints().entrySet().iterator().next()
+				.getKey();
+
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId1));
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId1));
+
+			CompletedCheckpoint latest = coord.getCheckpointStore().getLatestCheckpoint();
+
+			verifyExternalizedCheckpoint(latest, jid, checkpointId1, timestamp1);
+			verifyExternalizedCheckpointRestore(latest, jobVertices, vertex1, vertex2);
+		}
+
+		// ---------------
+		// trigger checkpoint 2
+		// ---------------
+
+		{
+			final long timestamp2 = System.currentTimeMillis() + 7;
+			coord.triggerCheckpoint(timestamp2, false);
+
+			long checkpointId2 = coord.getPendingCheckpoints().entrySet().iterator().next().getKey();
+
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId2));
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId2));
+
+			CompletedCheckpoint latest = coord.getCheckpointStore().getLatestCheckpoint();
+			verifyExternalizedCheckpoint(latest, jid, checkpointId2, timestamp2);
+			verifyExternalizedCheckpointRestore(latest, jobVertices, vertex1, vertex2);
+		}
+
+		// ---------------
+		// trigger checkpoint 3
+		// ---------------
+
+		{
+			final long timestamp3 = System.currentTimeMillis() + 146;
+			coord.triggerCheckpoint(timestamp3, false);
+
+			long checkpointId3 = coord.getPendingCheckpoints().entrySet().iterator().next().getKey();
+
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId3));
+			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId3));
+
+			CompletedCheckpoint latest = coord.getCheckpointStore().getLatestCheckpoint();
+			verifyExternalizedCheckpoint(latest, jid, checkpointId3, timestamp3);
+			verifyExternalizedCheckpointRestore(latest, jobVertices, vertex1, vertex2);
+		}
+
+		coord.shutdown(JobStatus.FINISHED);
+	}
+
+	/**
+	 * Verifies an externalized completed checkpoint instance.
+	 *
+	 * <p>The provided JobID, checkpoint ID, timestamp need to match. Also, the
+	 * external pointer and external metadata need to be notNull and exist (currently
+	 * assuming that they are file system based).
+	 *
+	 * @param checkpoint Completed checkpoint to check.
+	 * @param jid JobID of the job the checkpoint belongs to.
+	 * @param checkpointId Checkpoint ID of the checkpoint to check.
+	 * @param timestamp Timestamp of the checkpoint to check.
+	 */
+	private static void verifyExternalizedCheckpoint(CompletedCheckpoint checkpoint, JobID jid, long checkpointId, long timestamp) {
+		assertEquals(jid, checkpoint.getJobId());
+		assertEquals(checkpointId, checkpoint.getCheckpointID());
+		assertEquals(timestamp, checkpoint.getTimestamp());
+		assertNotNull(checkpoint.getExternalPointer());
+		assertNotNull(checkpoint.getExternalizedMetadata());
+		FileStateHandle fsHandle = (FileStateHandle) checkpoint.getExternalizedMetadata();
+		assertTrue(new File(fsHandle.getFilePath().getPath()).exists());
+	}
+
+	private static void verifyExternalizedCheckpointRestore(
+			CompletedCheckpoint checkpoint,
+			Map<JobVertexID, ExecutionJobVertex> jobVertices,
+			ExecutionVertex... vertices) throws IOException {
+
+		CompletedCheckpoint loaded = SavepointLoader.loadAndValidateSavepoint(
+				checkpoint.getJobId(),
+				jobVertices,
+				checkpoint.getExternalPointer(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		for (ExecutionVertex vertex : vertices) {
+			assertEquals(checkpoint.getTaskState(vertex.getJobvertexId()), loaded.getTaskState(vertex.getJobvertexId()));
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c477d87c/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
index d8e46fa..1691370 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
@@ -2586,7 +2586,7 @@ public class CheckpointCoordinatorTest {
 		return ChainedStateHandle.wrapSingleHandle(operatorStateHandle);
 	}
 
-	private static ExecutionJobVertex mockExecutionJobVertex(
+	static ExecutionJobVertex mockExecutionJobVertex(
 		JobVertexID jobVertexID,
 		int parallelism,
 		int maxParallelism) {

http://git-wip-us.apache.org/repos/asf/flink/blob/c477d87c/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java
index dc19e47..1eb8055 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java
@@ -19,12 +19,12 @@
 package org.apache.flink.runtime.checkpoint.savepoint;
 
 import java.io.File;
-import java.util.Arrays;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.TaskState;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -189,6 +189,27 @@ public class SavepointStoreTest {
 		assertEquals("Savepoint file not cleaned up on failure", 0, tmp.getRoot().listFiles().length);
 	}
 
+	/**
+	 * Tests that multiple externalized checkpoints can be stored to the same
+	 * directory.
+	 */
+	@Test
+	public void testStoreExternalizedCheckpointsToSameDirectory() throws Exception {
+		String root = tmp.newFolder().getAbsolutePath();
+		FileSystem fs = FileSystem.get(new Path(root).toUri());
+
+		// Store
+		SavepointV1 savepoint = new SavepointV1(1929292, SavepointV1Test.createTaskStates(4, 24));
+
+		FileStateHandle store1 = SavepointStore.storeExternalizedCheckpointToHandle(root, savepoint);
+		fs.exists(store1.getFilePath());
+		assertTrue(store1.getFilePath().getPath().contains(SavepointStore.EXTERNALIZED_CHECKPOINT_METADATA_FILE));
+
+		FileStateHandle store2 = SavepointStore.storeExternalizedCheckpointToHandle(root, savepoint);
+		fs.exists(store2.getFilePath());
+		assertTrue(store2.getFilePath().getPath().contains(SavepointStore.EXTERNALIZED_CHECKPOINT_METADATA_FILE));
+	}
+
 	private static class NewSavepointSerializer implements SavepointSerializer<TestSavepoint> {
 
 		private static final NewSavepointSerializer INSTANCE = new NewSavepointSerializer();


[02/11] flink git commit: [hotfix] [runtime] Reduce the gazillion deprecation warnings from the backwards-compatible savepoint path

Posted by se...@apache.org.
[hotfix] [runtime] Reduce the gazillion deprecation warnings from the backwards-compatible savepoint path


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

Branch: refs/heads/master
Commit: 40743aac9257677142a510fdd869036d13436edc
Parents: daf0ccd
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Feb 24 18:22:50 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Feb 28 18:59:09 2017 +0100

----------------------------------------------------------------------
 .../flink/migration/MigrationNamespaceSerializerProxy.java       | 1 +
 .../src/main/java/org/apache/flink/migration/MigrationUtil.java  | 1 +
 .../flink/migration/api/common/state/ListStateDescriptor.java    | 1 +
 .../apache/flink/migration/runtime/checkpoint/KeyGroupState.java | 1 +
 .../apache/flink/migration/runtime/checkpoint/SubtaskState.java  | 1 +
 .../org/apache/flink/migration/runtime/checkpoint/TaskState.java | 1 +
 .../migration/runtime/checkpoint/savepoint/SavepointV0.java      | 1 +
 .../runtime/checkpoint/savepoint/SavepointV0Serializer.java      | 1 +
 .../flink/migration/runtime/state/AbstractCloseableHandle.java   | 1 +
 .../flink/migration/runtime/state/AbstractStateBackend.java      | 1 +
 .../apache/flink/migration/runtime/state/KvStateSnapshot.java    | 1 +
 .../org/apache/flink/migration/runtime/state/StateHandle.java    | 1 +
 .../org/apache/flink/migration/runtime/state/StateObject.java    | 1 +
 .../apache/flink/migration/runtime/state/StreamStateHandle.java  | 1 +
 .../runtime/state/filesystem/AbstractFileStateHandle.java        | 1 +
 .../runtime/state/filesystem/AbstractFsStateSnapshot.java        | 1 +
 .../runtime/state/filesystem/FileSerializableStateHandle.java    | 1 +
 .../runtime/state/filesystem/FileStreamStateHandle.java          | 1 +
 .../flink/migration/runtime/state/filesystem/FsFoldingState.java | 1 +
 .../flink/migration/runtime/state/filesystem/FsListState.java    | 1 +
 .../migration/runtime/state/filesystem/FsReducingState.java      | 1 +
 .../flink/migration/runtime/state/filesystem/FsStateBackend.java | 1 +
 .../flink/migration/runtime/state/filesystem/FsValueState.java   | 1 +
 .../migration/runtime/state/memory/AbstractMemStateSnapshot.java | 1 +
 .../migration/runtime/state/memory/ByteStreamStateHandle.java    | 1 +
 .../flink/migration/runtime/state/memory/MemFoldingState.java    | 1 +
 .../flink/migration/runtime/state/memory/MemListState.java       | 1 +
 .../flink/migration/runtime/state/memory/MemReducingState.java   | 1 +
 .../flink/migration/runtime/state/memory/MemValueState.java      | 1 +
 .../migration/runtime/state/memory/SerializedStateHandle.java    | 1 +
 .../flink/migration/state/MigrationKeyGroupStateHandle.java      | 4 ++--
 .../apache/flink/migration/state/MigrationStreamStateHandle.java | 4 ++--
 .../flink/migration/streaming/runtime/tasks/StreamTaskState.java | 1 +
 .../migration/streaming/runtime/tasks/StreamTaskStateList.java   | 1 +
 34 files changed, 36 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
index 808a9ae..91fa633 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
@@ -32,6 +32,7 @@ import java.io.Serializable;
  * the first run under Flink 1.2 and provides again the real namespace serializer.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class MigrationNamespaceSerializerProxy extends TypeSerializer<Serializable> {
 
 	public static final MigrationNamespaceSerializerProxy INSTANCE = new MigrationNamespaceSerializerProxy();

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
index 76d4eef..9427f72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
@@ -25,6 +25,7 @@ import java.util.Collection;
 
 public class MigrationUtil {
 
+	@SuppressWarnings("deprecation")
 	public static boolean isOldSavepointKeyedState(Collection<KeyGroupsStateHandle> keyGroupsStateHandles) {
 		return (keyGroupsStateHandles != null)
 				&& (keyGroupsStateHandles.size() == 1)

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
index 3b1af54..1847c08 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
  */
 @Internal
 @Deprecated
+@SuppressWarnings("deprecation")
 public class ListStateDescriptor<T> extends StateDescriptor<ListState<T>, T> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
index ad94993..483a403 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
@@ -32,6 +32,7 @@ import java.io.Serializable;
  * which might not be available on the JobManager.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class KeyGroupState implements Serializable {
 	private static final long serialVersionUID = -5926696455438467634L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
index df886b3..7e84fdc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
@@ -26,6 +26,7 @@ import java.io.Serializable;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class SubtaskState implements Serializable {
 
 	private static final long serialVersionUID = -2394696997971923995L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
index 798c112..91d8ba0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
@@ -30,6 +30,7 @@ import java.util.Objects;
 import java.util.Set;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class TaskState implements Serializable {
 
 	private static final long serialVersionUID = -4845578005863201810L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
index 8aa562e..1c51a69 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
@@ -29,6 +29,7 @@ import java.util.Collection;
  *
  * <p>This format was introduced with Flink 1.1.0.
  */
+@SuppressWarnings("deprecation")
 public class SavepointV0 implements Savepoint {
 
 	/** The savepoint version. */

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
index 2efe786..4739033 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
@@ -67,6 +67,7 @@ import java.util.Map;
  * that no default Java serialization is used for serialization. Therefore, we
  * don't rely on any involved Java classes to stay the same.
  */
+@SuppressWarnings("deprecation")
 public class SavepointV0Serializer implements SavepointSerializer<SavepointV1> {
 
 	public static final SavepointV0Serializer INSTANCE = new SavepointV0Serializer();

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
index 873dab8..ad75dae 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
  * the handle is closed or was already closed.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public abstract class AbstractCloseableHandle implements Closeable, Serializable {
 
 	/** Serial Version UID must be constant to maintain format compatibility */

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
index 1d76c06..ff97e19 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
@@ -28,6 +28,7 @@ import java.io.Serializable;
  * A state backend defines how state is stored and snapshotted during checkpoints.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public abstract class AbstractStateBackend implements Serializable {
 	
 	private static final long serialVersionUID = 4620413814639220247L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
index 9936ca7..7aecd31 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public interface KvStateSnapshot<K, N, S extends State, SD extends StateDescriptor<S, ?>>
 		extends StateObject {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
index 97d6984..8cdeefa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
@@ -24,6 +24,7 @@ package org.apache.flink.migration.runtime.state;
  * is lightweight or fetching it lazily from some external storage when the state is too large.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public interface StateHandle<T> extends StateObject {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
index 2f1048f..128a074 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
@@ -34,6 +34,7 @@ package org.apache.flink.migration.runtime.state;
  * </ul>
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public interface StateObject extends java.io.Closeable, java.io.Serializable {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
index 55f9b58..ee6be97 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
@@ -25,6 +25,7 @@ import java.io.Serializable;
  * A state handle that produces an input stream when resolved.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public interface StreamStateHandle extends StateHandle<InputStream> {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
index fcdcc78..3d77064 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
@@ -32,6 +32,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * Base class for state that is stored in a file.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public abstract class AbstractFileStateHandle extends AbstractCloseableHandle implements StateObject {
 
 	private static final long serialVersionUID = 350284443258002355L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
index 59c373b..103c214 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
@@ -34,6 +34,7 @@ import java.io.IOException;
  * @param <SV> The type of the state value.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public abstract class AbstractFsStateSnapshot<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>> 
 		extends AbstractFileStateHandle implements KvStateSnapshot<K, N, S, SD> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
index ef908f5..bee5ded 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
@@ -33,6 +33,7 @@ import java.io.Serializable;
  * @param <T> The type of state pointed to by the state handle.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FileSerializableStateHandle<T extends Serializable> extends AbstractFileStateHandle implements StateHandle<T> {
 
 	private static final long serialVersionUID = -657631394290213622L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
index 89ff4c4..e44b823 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
@@ -30,6 +30,7 @@ import java.io.Serializable;
  * A state handle that points to state in a file system, accessible as an input stream.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FileStreamStateHandle extends AbstractFileStateHandle implements StreamStateHandle {
 
 	private static final long serialVersionUID = -6826990484549987311L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
index e1bac83..6e98a30 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.fs.Path;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FsFoldingState<K, N, T, ACC> {
 	public static class Snapshot<K, N, T, ACC> extends AbstractFsStateSnapshot<K, N, ACC, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
index d4e3d4b..e8f7a9d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
@@ -26,6 +26,7 @@ import org.apache.flink.core.fs.Path;
 import java.util.ArrayList;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FsListState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, ArrayList<V>, ListState<V>, ListStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
index 5cd9505..8174871 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.fs.Path;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FsReducingState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, V, ReducingState<V>, ReducingStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
index e964ec9..fbdd393 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FsStateBackend extends AbstractStateBackend {
 
 	private static final long serialVersionUID = -8191916350224044011L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
index 3b432a3..b185f88 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.fs.Path;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class FsValueState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, V, ValueState<V>, ValueStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
index 3336556..6056578 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
@@ -29,6 +29,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public abstract class AbstractMemStateSnapshot<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>> 
 		implements KvStateSnapshot<K, N, S, SD> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
index d9474dc..45785bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
@@ -28,6 +28,7 @@ import java.io.InputStream;
 import java.io.Serializable;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public final class ByteStreamStateHandle extends AbstractCloseableHandle implements StreamStateHandle {
 
 	private static final long serialVersionUID = -5280226231200217594L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
index d6c63c4..0956ddd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class MemFoldingState<K, N, T, ACC> {
 
 	public static class Snapshot<K, N, T, ACC> extends AbstractMemStateSnapshot<K, N, ACC, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
index 416a898..3ff7c50 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import java.util.ArrayList;
 
 @Deprecated
+@SuppressWarnings("deprecation")
 public class MemListState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, ArrayList<V>, ListState<V>, ListStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
index 52d60a9..e6154fb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
  * @param <V> The type of the values in the list state.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class MemReducingState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, V, ReducingState<V>, ReducingStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
index ff9bed8..777b0f6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
@@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
  * @param <V> The type of the value.
  */
 @Deprecated
+@SuppressWarnings("deprecation")
 public class MemValueState<K, N, V> {
 
 	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, V, ValueState<V>, ValueStateDescriptor<V>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
index d3c9b6c..49d772e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
@@ -31,6 +31,7 @@ import java.io.Serializable;
  *
  * @param <T> The type of state represented by this state handle.
  */
+@SuppressWarnings("deprecation")
 public class SerializedStateHandle<T extends Serializable> extends AbstractCloseableHandle implements StateHandle<T> {
 	
 	private static final long serialVersionUID = 4145685722538475769L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
index 995d234..ee15b83 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
@@ -24,12 +24,12 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.util.Migration;
 
-@Internal
-@Deprecated
 /**
  * This class is just a KeyGroupsStateHandle that is tagged as migration, to figure out which restore logic to apply,
  * e.g. when restoring backend data from a state handle.
  */
+@Internal
+@Deprecated
 public class MigrationKeyGroupStateHandle extends KeyGroupsStateHandle implements Migration {
 
 	private static final long serialVersionUID = -8554427169776881697L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
index e7831a9..797d5d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
@@ -26,12 +26,12 @@ import org.apache.flink.util.Migration;
 
 import java.io.IOException;
 
-@Internal
-@Deprecated
 /**
  * This class is just a StreamStateHandle that is tagged as migration, to figure out which restore logic to apply, e.g.
  * when restoring backend data from a state handle.
  */
+@Internal
+@Deprecated
 public class MigrationStreamStateHandle implements StreamStateHandle, Migration {
 
 	private static final long serialVersionUID = -2332113722532150112L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
index f5af185..a6b6e3f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
@@ -29,6 +29,7 @@ import java.util.HashMap;
 
 @Deprecated
 @Internal
+@SuppressWarnings("deprecation")
 public class StreamTaskState implements Serializable, Closeable {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/40743aac/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
index 8b0dcd3..07ce44b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
@@ -27,6 +27,7 @@ import java.util.HashMap;
 
 @Deprecated
 @Internal
+@SuppressWarnings("deprecation")
 public class StreamTaskStateList implements StateHandle<StreamTaskState[]> {
 
 	private static final long serialVersionUID = 1L;