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 2018/01/18 17:09:18 UTC

[01/17] flink git commit: [FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata

Repository: flink
Updated Branches:
  refs/heads/master 03c797a1d -> 1f9c2d974


http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackendTest.java
new file mode 100644
index 0000000..4aac253
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackendTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nullable;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link AbstractFileStateBackend}.
+ */
+public class AbstractFileStateBackendTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+	//  pointers
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testPointerPathResolution() throws Exception {
+		final FileSystem fs = FileSystem.getLocalFileSystem();
+
+		final Path checkpointDir = new Path(tmp.newFolder().toURI());
+		final Path metadataFile = new Path(checkpointDir, AbstractFsCheckpointStorage.METADATA_FILE_NAME);
+
+		final String pointer1 = metadataFile.toString();
+		final String pointer2 = metadataFile.getParent().toString();
+		final String pointer3 = metadataFile.getParent().toString() + '/';
+
+		final FsStateBackend backend = new FsStateBackend(checkpointDir);
+
+		final byte[] data = new byte[23686];
+		new Random().nextBytes(data);
+		try (FSDataOutputStream out = fs.create(metadataFile, WriteMode.NO_OVERWRITE)) {
+			out.write(data);
+		}
+
+		StreamStateHandle handle1 = backend.resolveCheckpoint(pointer1);
+		StreamStateHandle handle2 = backend.resolveCheckpoint(pointer2);
+		StreamStateHandle handle3 = backend.resolveCheckpoint(pointer3);
+
+		assertNotNull(handle1);
+		assertNotNull(handle2);
+		assertNotNull(handle3);
+
+		validateContents(handle1, data);
+		validateContents(handle2, data);
+		validateContents(handle3, data);
+	}
+
+	@Test
+	public void testFailingPointerPathResolution() throws Exception {
+		final Path checkpointDir = new Path(tmp.newFolder().toURI());
+		final FsStateBackend backend = new FsStateBackend(checkpointDir);
+
+		// null value
+		try {
+			backend.resolveCheckpoint(null);
+			fail("expected exception");
+		} catch (NullPointerException ignored) {}
+
+		// empty string
+		try {
+			backend.resolveCheckpoint("");
+			fail("expected exception");
+		} catch (IllegalArgumentException ignored) {}
+
+		// not a file path at all
+		try {
+			backend.resolveCheckpoint("this-is_not/a#filepath.at.all");
+			fail("expected exception");
+		} catch (IOException ignored) {}
+
+		// non-existing file
+		try {
+			backend.resolveCheckpoint(tmp.newFile().toURI().toString() + "_not_existing");
+			fail("expected exception");
+		} catch (IOException ignored) {}
+	}
+
+	// ------------------------------------------------------------------------
+	//  checkpoints
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Validates that multiple checkpoints from different jobs with the same checkpoint ID do not
+	 * interfere with each other.
+	 */
+	@Test
+	public void testPersistMultipleMetadataOnlyCheckpoints() throws Exception {
+		final FileSystem fs = FileSystem.getLocalFileSystem();
+		final Path checkpointDir = new Path(tmp.newFolder().toURI());
+
+		final FsStateBackend backend = new FsStateBackend(checkpointDir);
+
+		final JobID jobId1 = new JobID();
+		final JobID jobId2 = new JobID();
+
+		final long checkpointId = 177;
+
+		final CheckpointStorage storage1 = backend.createCheckpointStorage(jobId1);
+		final CheckpointStorage storage2 = backend.createCheckpointStorage(jobId2);
+
+		final CheckpointStorageLocation loc1 = storage1.initializeLocationForCheckpoint(checkpointId);
+		final CheckpointStorageLocation loc2 = storage2.initializeLocationForCheckpoint(checkpointId);
+
+		final byte[] data1 = {77, 66, 55, 99, 88};
+		final byte[] data2 = {1, 3, 2, 5, 4};
+
+		try (CheckpointStateOutputStream out = loc1.createMetadataOutputStream()) {
+			out.write(data1);
+			out.closeAndGetHandle();
+		}
+		final String result1 = loc1.markCheckpointAsFinished();
+
+		try (CheckpointStateOutputStream out = loc2.createMetadataOutputStream()) {
+			out.write(data2);
+			out.closeAndGetHandle();
+		}
+		final String result2 = loc2.markCheckpointAsFinished();
+
+		// check that this went to a file, but in a nested directory structure
+
+		// one directory per job
+		FileStatus[] files = fs.listStatus(checkpointDir);
+		assertEquals(2, files.length);
+
+		// in each per-job directory, one for the checkpoint
+		FileStatus[] job1Files = fs.listStatus(files[0].getPath());
+		FileStatus[] job2Files = fs.listStatus(files[1].getPath());
+		assertEquals(3, job1Files.length);
+		assertEquals(3, job2Files.length);
+
+		assertTrue(fs.exists(new Path(result1, AbstractFsCheckpointStorage.METADATA_FILE_NAME)));
+		assertTrue(fs.exists(new Path(result2, AbstractFsCheckpointStorage.METADATA_FILE_NAME)));
+
+		validateContents(backend.resolveCheckpoint(result1), data1);
+		validateContents(backend.resolveCheckpoint(result2), data2);
+	}
+
+	@Test
+	public void writeToAlreadyExistingCheckpointFails() throws Exception {
+		final Path checkpointDir = new Path(tmp.newFolder().toURI());
+		final FsStateBackend backend = new FsStateBackend(checkpointDir);
+
+		final JobID jobId = new JobID();
+		final byte[] data = {8, 8, 4, 5, 2, 6, 3};
+		final long checkpointId = 177;
+
+		final CheckpointStorage storage = backend.createCheckpointStorage(jobId);
+		final CheckpointStorageLocation loc = storage.initializeLocationForCheckpoint(checkpointId);
+
+		// write to the metadata file for the checkpoint
+
+		try (CheckpointStateOutputStream out = loc.createMetadataOutputStream()) {
+			out.write(data);
+			out.closeAndGetHandle();
+		}
+
+		// create another writer to the metadata file for the checkpoint
+		try {
+			loc.createMetadataOutputStream();
+			fail("this should fail with an exception");
+		}
+		catch (IOException ignored) {}
+	}
+
+	// ------------------------------------------------------------------------
+	//  savepoints
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testSavepointPathConfiguredAndTarget() throws Exception {
+		final Path savepointDir = Path.fromLocalFile(tmp.newFolder());
+		final Path customDir = Path.fromLocalFile(tmp.newFolder());
+		testSavepoint(savepointDir, customDir, customDir);
+	}
+
+	@Test
+	public void testSavepointPathConfiguredNoTarget() throws Exception {
+		final Path savepointDir = Path.fromLocalFile(tmp.newFolder());
+		testSavepoint(savepointDir, null, savepointDir);
+	}
+
+	@Test
+	public void testNoSavepointPathConfiguredAndTarget() throws Exception {
+		final Path customDir = Path.fromLocalFile(tmp.newFolder());
+		testSavepoint(null, customDir, customDir);
+	}
+
+	@Test
+	public void testNoSavepointPathConfiguredNoTarget() throws Exception {
+		final Path checkpointDir = Path.fromLocalFile(tmp.newFolder());
+		final CheckpointStorage storage = new FsStateBackend(checkpointDir.toUri(), null)
+				.createCheckpointStorage(new JobID());
+
+		try {
+			storage.initializeLocationForSavepoint(1337, null);
+			fail("this should fail with an exception");
+		}
+		catch (IllegalArgumentException ignored) {}
+	}
+
+	private void testSavepoint(
+			@Nullable Path savepointDir,
+			@Nullable Path customDir,
+			Path expectedParent) throws Exception {
+
+		final JobID jobId = new JobID();
+
+		final FsCheckpointStorage storage = (FsCheckpointStorage)
+				new FsStateBackend(tmp.newFolder().toURI(), savepointDir == null ? null : savepointDir.toUri())
+				.createCheckpointStorage(jobId);
+
+		final String customLocation = customDir == null ? null : customDir.toString();
+
+		final FsCheckpointStorageLocation savepointLocation =
+				storage.initializeLocationForSavepoint(52452L, customLocation);
+
+		// all state types should be in the expected location
+		assertParent(expectedParent, savepointLocation.getCheckpointDirectory());
+		assertParent(expectedParent, savepointLocation.getSharedStateDirectory());
+		assertParent(expectedParent, savepointLocation.getTaskOwnedStateDirectory());
+
+		final byte[] data = {77, 66, 55, 99, 88};
+
+		final StreamStateHandle handle;
+		try (CheckpointStateOutputStream out = savepointLocation.createMetadataOutputStream()) {
+			out.write(data);
+			handle = out.closeAndGetHandle();
+		}
+		validateContents(handle, data);
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static void validateContents(StreamStateHandle handle, byte[] expected) throws IOException {
+		try (FSDataInputStream in = handle.openInputStream()) {
+			validateContents(in, expected);
+		}
+	}
+
+	private static void validateContents(InputStream in, byte[] expected) throws IOException {
+		final byte[] buffer = new byte[expected.length];
+
+		int pos = 0;
+		int remaining = expected.length;
+		while (remaining > 0) {
+			int read = in.read(buffer, pos, remaining);
+			if (read == -1) {
+				throw new EOFException();
+			}
+			pos += read;
+			remaining -= read;
+		}
+
+		assertArrayEquals(expected, buffer);
+	}
+
+	private void assertParent(Path parent, Path child) {
+		Path path = new Path(parent, child.getName());
+		assertEquals(path, child);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStreamTest.java
new file mode 100644
index 0000000..869cc00
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStreamTest.java
@@ -0,0 +1,285 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.core.fs.FSDataInputStream;
+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 org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.core.testutils.OneShotLatch;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.mockito.ArgumentCaptor;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link }.
+ */
+public class FixFileFsStateOutputStreamTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+	//  Tests
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Validates that even empty streams create a file and a file state handle.
+	 */
+	@Test
+	public void testEmptyState() throws Exception {
+		final FileSystem fs = FileSystem.getLocalFileSystem();
+		final Path path = new Path(new Path(tmp.newFolder().toURI()), "myFileName");
+
+		final FileStateHandle handle;
+		try (FixFileFsStateOutputStream stream = new FixFileFsStateOutputStream(fs, path)) {
+			handle = stream.closeAndGetHandle();
+		}
+
+		// must have created a handle
+		assertNotNull(handle);
+		assertEquals(path, handle.getFilePath());
+
+		// the pointer path should exist as a directory
+		assertTrue(fs.exists(handle.getFilePath()));
+		assertFalse(fs.getFileStatus(path).isDir());
+
+		// the contents should be empty
+		try (FSDataInputStream in = handle.openInputStream()) {
+			assertEquals(-1, in.read());
+		}
+	}
+
+	/**
+	 * Simple write and read test
+	 */
+	@Test
+	public void testWriteAndRead() throws Exception {
+		final FileSystem fs = FileSystem.getLocalFileSystem();
+		final Path path = new Path(new Path(tmp.newFolder().toURI()), "fooBarName");
+
+		final Random rnd = new Random();
+		final byte[] data = new byte[1694523];
+
+		// write the data (mixed single byte writes and array writes)
+		final FileStateHandle handle;
+		try (FixFileFsStateOutputStream stream = new FixFileFsStateOutputStream(fs, path)) {
+			for (int i = 0; i < data.length;) {
+				if (rnd.nextBoolean()) {
+					stream.write(data[i++]);
+				}
+				else {
+					int len = rnd.nextInt(Math.min(data.length - i, 32));
+					stream.write(data, i, len);
+					i += len;
+				}
+			}
+			handle = stream.closeAndGetHandle();
+		}
+
+		// (1) stream from handle must hold the contents
+		try (FSDataInputStream in = handle.openInputStream()) {
+			byte[] buffer = new byte[data.length];
+			readFully(in, buffer);
+			assertArrayEquals(data, buffer);
+		}
+
+		// (2) the pointer must point to a file with that contents
+		try (FSDataInputStream in = fs.open(handle.getFilePath())) {
+			byte[] buffer = new byte[data.length];
+			readFully(in, buffer);
+			assertArrayEquals(data, buffer);
+		}
+	}
+
+	/**
+	 * Tests that the underlying stream file is deleted upon calling close.
+	 */
+	@Test
+	public void testCleanupWhenClosingStream() throws IOException {
+		final FileSystem fs = FileSystem.getLocalFileSystem();
+		final Path path = new Path(new Path(tmp.newFolder().toURI()), "nonCreativeTestFileName");
+
+		// write some test data and close the stream
+		try (FixFileFsStateOutputStream stream = new FixFileFsStateOutputStream(fs, path)) {
+			Random rnd = new Random();
+			for (int i = 0; i < rnd.nextInt(1000); i++) {
+				stream.write(rnd.nextInt(100));
+			}
+			assertTrue(fs.exists(path));
+		}
+
+		assertFalse(fs.exists(path));
+	}
+
+	/**
+	 * Tests that the underlying stream file is deleted if the closeAndGetHandle method fails.
+	 */
+	@Test
+	public void testCleanupWhenFailingCloseAndGetHandle() throws IOException {
+		final Path path = new Path(new Path(tmp.newFolder().toURI()), "neverCreated");
+
+		final FileSystem fs = mock(FileSystem.class);
+		final FSDataOutputStream outputStream = mock(FSDataOutputStream.class);
+
+		final ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
+
+		when(fs.create(pathCaptor.capture(), any(WriteMode.class))).thenReturn(outputStream);
+		doThrow(new IOException("Test IOException.")).when(outputStream).close();
+
+		FixFileFsStateOutputStream stream = new FixFileFsStateOutputStream(fs, path);
+		verify(fs).create(any(Path.class), any(WriteMode.class));
+
+		stream.write(new byte[] {1,2,3,4,5});
+
+		try {
+			stream.closeAndGetHandle();
+			fail("Expected IOException");
+		}
+		catch (IOException ignored) {
+			// expected exception
+		}
+
+		verify(fs).delete(eq(pathCaptor.getValue()), anyBoolean());
+	}
+
+	/**
+	 * This test validates that a close operation can happen even while a 'closeAndGetHandle()'
+	 * call is in progress.
+	 *
+	 * <p>That behavior is essential for fast cancellation (concurrent cleanup).
+	 */
+	@Test
+	public void testCloseDoesNotLock() throws Exception {
+		// a stream that blocks but is released when closed
+		final FSDataOutputStream stream = new BlockerStream();
+
+		final FileSystem fileSystem = mock(FileSystem.class);
+		when(fileSystem.create(any(Path.class), any(WriteMode.class))).thenReturn(stream);
+
+		final Path path = new Path(new Path(tmp.newFolder().toURI()), "this-is-ignored-anyways.file");
+		final FixFileFsStateOutputStream checkpointStream =
+				new FixFileFsStateOutputStream(fileSystem, path);
+
+		final OneShotLatch sync = new OneShotLatch();
+
+		final CheckedThread thread = new CheckedThread() {
+
+			@Override
+			public void go() throws Exception {
+				sync.trigger();
+				// that call should now block, because it accesses the position
+				checkpointStream.closeAndGetHandle();
+			}
+		};
+		thread.start();
+
+		sync.await();
+		checkpointStream.close();
+
+		// the thread may or may not fail, that depends on the thread race
+		// it is not important for this test, important is that the thread does not freeze/lock up
+		try {
+			thread.sync();
+		} catch (IOException ignored) {}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static void readFully(InputStream in, byte[] buffer) throws IOException {
+		int pos = 0;
+		int remaining = buffer.length;
+
+		while (remaining > 0) {
+			int read = in.read(buffer, pos, remaining);
+			if (read == -1) {
+				throw new EOFException();
+			}
+
+			pos += read;
+			remaining -= read;
+		}
+	}
+
+	private static class BlockerStream extends FSDataOutputStream {
+
+		private final OneShotLatch blocker = new OneShotLatch();
+
+		@Override
+		public long getPos() throws IOException {
+			block();
+			return 0L;
+		}
+
+		@Override
+		public void write(int b) throws IOException {
+			block();
+		}
+
+		@Override
+		public void flush() throws IOException {
+			block();
+		}
+
+		@Override
+		public void sync() throws IOException {
+			block();
+		}
+
+		@Override
+		public void close() throws IOException {
+			blocker.trigger();
+		}
+
+		private void block() throws IOException {
+			try {
+				blocker.await();
+			} catch (InterruptedException e) {
+				throw new IOException("interrupted");
+			}
+			throw new IOException("closed");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/state/memory/MemoryCheckpointStorageTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/memory/MemoryCheckpointStorageTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/memory/MemoryCheckpointStorageTest.java
new file mode 100644
index 0000000..5820b63
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/memory/MemoryCheckpointStorageTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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.memory;
+
+/**
+ * Tests for the checkpoint storage aspects of the {@link MemoryStateBackend}.
+ */
+public class MemoryCheckpointStorageTest {
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 61e83be..b33a78e 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
@@ -25,10 +25,10 @@ import akka.testkit.{ImplicitSender, TestKit}
 import akka.util.Timeout
 import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.akka.ListeningBehaviour
-import org.apache.flink.runtime.checkpoint.{CheckpointCoordinator, CompletedCheckpoint}
+import org.apache.flink.runtime.checkpoint.{CheckpointRetentionPolicy, CheckpointCoordinator, CompletedCheckpoint}
 import org.apache.flink.runtime.client.JobExecutionException
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType
-import org.apache.flink.runtime.jobgraph.tasks.{ExternalizedCheckpointSettings, CheckpointCoordinatorConfiguration, JobCheckpointingSettings}
+import org.apache.flink.runtime.jobgraph.tasks.{CheckpointCoordinatorConfiguration, JobCheckpointingSettings}
 import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobGraph, JobVertex, ScheduleMode}
 import org.apache.flink.runtime.jobmanager.Tasks._
 import org.apache.flink.runtime.jobmanager.scheduler.{NoResourceAvailableException, SlotSharingGroup}
@@ -812,7 +812,7 @@ class JobManagerITCase(_system: ActorSystem)
           // Verify the response
           response.jobId should equal(jobGraph.getJobID())
           response.cause.getClass should equal(classOf[IllegalStateException])
-          response.cause.getMessage should (include("disabled") or include("configured"))
+          response.cause.getMessage should include("not a streaming job")
         }
       }
       finally {
@@ -842,7 +842,7 @@ class JobManagerITCase(_system: ActorSystem)
               60000,
               60000,
               1,
-              ExternalizedCheckpointSettings.none,
+              CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
               true),
             null))
 
@@ -903,7 +903,7 @@ class JobManagerITCase(_system: ActorSystem)
               60000,
               60000,
               1,
-              ExternalizedCheckpointSettings.none,
+              CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
               true),
             null))
 
@@ -972,7 +972,7 @@ class JobManagerITCase(_system: ActorSystem)
               60000,
               60000,
               1,
-              ExternalizedCheckpointSettings.none,
+              CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
               true),
             null))
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
index cd88133..6e1c7ee 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
@@ -18,15 +18,17 @@
 
 package org.apache.flink.runtime.testingUtils
 
+import java.io.DataInputStream
 import java.util.function.BiFunction
 
 import akka.actor.{ActorRef, Cancellable, Terminated}
 import akka.pattern.{ask, pipe}
 import org.apache.flink.api.common.JobID
+import org.apache.flink.core.fs.FSDataInputStream
 import org.apache.flink.runtime.FlinkActor
 import org.apache.flink.runtime.checkpoint.CheckpointOptions.CheckpointType
-import org.apache.flink.runtime.checkpoint.CompletedCheckpoint
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
+import org.apache.flink.runtime.checkpoint.savepoint.Savepoint
+import org.apache.flink.runtime.checkpoint.{Checkpoints, CompletedCheckpoint}
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.jobgraph.JobStatus
 import org.apache.flink.runtime.jobmanager.JobManager
@@ -37,6 +39,8 @@ import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.messages.Messages.Disconnect
 import org.apache.flink.runtime.messages.RegistrationMessages.RegisterTaskManager
 import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat
+import org.apache.flink.runtime.state.memory.MemoryStateBackend
+import org.apache.flink.runtime.state.{StateBackend, StateBackendLoader, StreamStateHandle}
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingMessages._
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged
@@ -314,11 +318,22 @@ trait TestingJobManagerLike extends FlinkActor {
 
     case RequestSavepoint(savepointPath) =>
       try {
-        //TODO user class loader ?
-        val savepoint = SavepointStore.loadSavepoint(
-          savepointPath,
-          Thread.currentThread().getContextClassLoader)
-        
+        val classloader = Thread.currentThread().getContextClassLoader
+
+        val loadedBackend = StateBackendLoader.loadStateBackendFromConfig(
+          flinkConfiguration, classloader, null)
+        val backend = if (loadedBackend != null) loadedBackend else new MemoryStateBackend()
+
+        val metadataHandle = backend.resolveCheckpoint(savepointPath)
+
+        val stream = new DataInputStream(metadataHandle.openInputStream())
+        val savepoint = try {
+          Checkpoints.loadCheckpointMetadata(stream, classloader)
+        }
+        finally {
+          stream.close()
+        }
+
         sender ! ResponseSavepoint(savepoint)
       }
       catch {

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 fce9dc9..aa9c685 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
@@ -27,6 +27,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -39,7 +40,6 @@ import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
@@ -615,16 +615,18 @@ public class StreamingJobGraphGenerator {
 
 		//  --- configure options ---
 
-		ExternalizedCheckpointSettings externalizedCheckpointSettings;
+		CheckpointRetentionPolicy retentionAfterTermination;
 		if (cfg.isExternalizedCheckpointsEnabled()) {
 			CheckpointConfig.ExternalizedCheckpointCleanup cleanup = cfg.getExternalizedCheckpointCleanup();
 			// Sanity check
 			if (cleanup == null) {
 				throw new IllegalStateException("Externalized checkpoints enabled, but no cleanup mode configured.");
 			}
-			externalizedCheckpointSettings = ExternalizedCheckpointSettings.externalizeCheckpoints(cleanup.deleteOnCancellation());
+			retentionAfterTermination = cleanup.deleteOnCancellation() ?
+					CheckpointRetentionPolicy.RETAIN_ON_FAILURE :
+					CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION;
 		} else {
-			externalizedCheckpointSettings = ExternalizedCheckpointSettings.none();
+			retentionAfterTermination = CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION;
 		}
 
 		CheckpointingMode mode = cfg.getCheckpointingMode();
@@ -697,7 +699,7 @@ public class StreamingJobGraphGenerator {
 				cfg.getCheckpointTimeout(),
 				cfg.getMinPauseBetweenCheckpoints(),
 				cfg.getMaxConcurrentCheckpoints(),
-				externalizedCheckpointSettings,
+				retentionAfterTermination,
 				isExactlyOnce),
 			serializedStateBackend,
 			serializedHooks);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java
index bff98fb..812cb56 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java
@@ -53,11 +53,13 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
@@ -109,7 +111,7 @@ public class StreamTaskTerminationTest extends TestLogger {
 		final StreamConfig streamConfig = new StreamConfig(taskConfiguration);
 		final NoOpStreamOperator<Long> noOpStreamOperator = new NoOpStreamOperator<>();
 
-		final AbstractStateBackend blockingStateBackend = new BlockingStateBackend();
+		final StateBackend blockingStateBackend = new BlockingStateBackend();
 
 		streamConfig.setStreamOperator(noOpStreamOperator);
 		streamConfig.setOperatorID(new OperatorID());
@@ -245,11 +247,21 @@ public class StreamTaskTerminationTest extends TestLogger {
 		private static final long serialVersionUID = 4517845269225218312L;
 	}
 
-	static class BlockingStateBackend extends AbstractStateBackend {
+	static class BlockingStateBackend implements StateBackend {
 
 		private static final long serialVersionUID = -5053068148933314100L;
 
 		@Override
+		public StreamStateHandle resolveCheckpoint(String pointer) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
 		public CheckpointStreamFactory createStreamFactory(JobID jobId, String operatorIdentifier) throws IOException {
 			return mock(CheckpointStreamFactory.class);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
index 2051771..b04a72a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
@@ -257,7 +257,7 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 	// ------------------------------------------------------------------------
 	private static class TestDeclinedCheckpointResponder implements CheckpointResponder {
 
-		OneShotLatch declinedLatch = new OneShotLatch();
+		final OneShotLatch declinedLatch = new OneShotLatch();
 
 		@Override
 		public void acknowledgeCheckpoint(

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 2711870..888c418 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
@@ -235,15 +235,6 @@ public class SavepointITCase extends TestLogger {
 				fail("Savepoint not created in expected directory");
 			}
 
-			// 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 clean: " + Arrays.toString(files), 0, files.length);
-			}
-
 			// - Verification END ---------------------------------------------
 
 			// Restart the cluster

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
index b694e0c..2882504 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
@@ -37,13 +37,11 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.util.TestBaseUtils;
 
 import org.apache.commons.io.FileUtils;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 ad0968a..21ce77a 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
@@ -29,10 +29,13 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.MemoryBackendCheckpointStorage;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.util.AbstractTestBase;
 
@@ -50,8 +53,6 @@ public class StateBackendITCase extends AbstractTestBase {
 
 	/**
 	 * Verify that the user-specified state backend is used even if checkpointing is disabled.
-	 *
-	 * @throws Exception
 	 */
 	@Test
 	public void testStateBackendWithoutCheckpointing() throws Exception {
@@ -70,7 +71,7 @@ public class StateBackendITCase extends AbstractTestBase {
 				@Override
 				public void open(Configuration parameters) throws Exception {
 					super.open(parameters);
-					getRuntimeContext().getState(new ValueStateDescriptor<Integer>("Test", Integer.class, 0));
+					getRuntimeContext().getState(new ValueStateDescriptor<>("Test", Integer.class));
 				}
 
 				@Override
@@ -90,10 +91,20 @@ public class StateBackendITCase extends AbstractTestBase {
 		}
 	}
 
-	private static class FailingStateBackend extends AbstractStateBackend {
+	private static class FailingStateBackend implements StateBackend {
 		private static final long serialVersionUID = 1L;
 
 		@Override
+		public StreamStateHandle resolveCheckpoint(String pointer) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+			return new MemoryBackendCheckpointStorage(jobId);
+		}
+
+		@Override
 		public CheckpointStreamFactory createStreamFactory(JobID jobId,
 				String operatorIdentifier) throws IOException {
 			throw new SuccessException();


[05/17] flink git commit: [FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata

Posted by se...@apache.org.
[FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata


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

Branch: refs/heads/master
Commit: edc6f1000704a492629d7bdf8cbfa5ba5c45bb1f
Parents: d19525e
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Oct 26 21:26:00 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../streaming/state/RocksDBStateBackend.java    |  26 +-
 .../jobmanager/JMXJobManagerMetricTest.java     |   7 +-
 .../checkpoint/CheckpointCoordinator.java       | 186 +++++------
 .../checkpoint/CheckpointProperties.java        | 105 ++----
 .../checkpoint/CheckpointRetentionPolicy.java   |  37 +++
 .../flink/runtime/checkpoint/Checkpoints.java   | 327 ++++++++++++++++++
 .../runtime/checkpoint/CompletedCheckpoint.java |  85 ++---
 .../runtime/checkpoint/PendingCheckpoint.java   | 149 +++------
 .../runtime/checkpoint/savepoint/Savepoint.java |   3 +-
 .../checkpoint/savepoint/SavepointLoader.java   | 159 ---------
 .../checkpoint/savepoint/SavepointStore.java    | 328 -------------------
 .../runtime/executiongraph/ExecutionGraph.java  |   8 +-
 .../executiongraph/ExecutionGraphBuilder.java   |  11 +-
 .../CheckpointCoordinatorConfiguration.java     |  27 +-
 .../tasks/ExternalizedCheckpointSettings.java   |  89 -----
 .../checkpoints/CheckpointConfigHandler.java    |   8 +-
 .../checkpoints/CheckpointConfigHandler.java    |   9 +-
 .../flink/runtime/state/CheckpointStorage.java  |  93 ++++++
 .../state/CheckpointStorageLocation.java        |  65 ++++
 .../runtime/state/CheckpointStreamFactory.java  |  27 ++
 .../flink/runtime/state/StateBackend.java       |  65 ++--
 .../filesystem/AbstractFileStateBackend.java    |  33 +-
 .../filesystem/AbstractFsCheckpointStorage.java | 256 +++++++++++++++
 .../filesystem/FixFileFsStateOutputStream.java  | 154 +++++++++
 .../state/filesystem/FsCheckpointStorage.java   |  85 +++++
 .../filesystem/FsCheckpointStorageLocation.java | 122 +++++++
 .../state/filesystem/FsStateBackend.java        |  11 +
 .../memory/MemoryBackendCheckpointStorage.java  | 135 ++++++++
 .../state/memory/MemoryStateBackend.java        |  24 +-
 ...istentMetadataCheckpointStorageLocation.java |  56 ++++
 ...istentMetadataCheckpointStorageLocation.java |  64 ++++
 .../flink/runtime/jobmanager/JobManager.scala   | 102 +++---
 ...tCoordinatorExternalizedCheckpointsTest.java | 206 ------------
 .../CheckpointCoordinatorFailureTest.java       |   4 +-
 .../CheckpointCoordinatorMasterHooksTest.java   |  20 +-
 .../checkpoint/CheckpointCoordinatorTest.java   | 116 +++----
 .../CheckpointExternalResumeTest.java           | 203 ++++++++++++
 .../CheckpointMetadataLoadingTest.java          | 132 ++++++++
 .../checkpoint/CheckpointPropertiesTest.java    |  38 +--
 .../CheckpointSettingsSerializableTest.java     |  28 +-
 .../checkpoint/CheckpointStateRestoreTest.java  |  25 +-
 .../checkpoint/CheckpointStatsHistoryTest.java  |   8 +-
 .../checkpoint/CheckpointStatsSnapshotTest.java |   6 +-
 .../checkpoint/CheckpointStatsTrackerTest.java  |  28 +-
 .../CompletedCheckpointStatsSummaryTest.java    |   4 +-
 .../CompletedCheckpointStoreTest.java           |   6 +-
 .../checkpoint/CompletedCheckpointTest.java     |  51 +--
 .../checkpoint/CoordinatorShutdownTest.java     |   7 +-
 ...ExecutionGraphCheckpointCoordinatorTest.java |   4 +-
 .../checkpoint/FailedCheckpointStatsTest.java   |   5 +-
 .../checkpoint/PendingCheckpointStatsTest.java  |   8 +-
 .../checkpoint/PendingCheckpointTest.java       | 143 ++++----
 .../checkpoint/RestoredCheckpointStatsTest.java |   2 +-
 .../ZooKeeperCompletedCheckpointStoreTest.java  |  13 +-
 .../checkpoint/hooks/MasterHooksTest.java       |   2 +
 .../savepoint/SavepointLoaderTest.java          | 124 -------
 .../savepoint/SavepointStoreTest.java           | 308 -----------------
 .../ArchivedExecutionGraphTest.java             |  13 +-
 .../ExecutionGraphDeploymentTest.java           |   4 +-
 .../IndividualRestartsConcurrencyTest.java      |   9 +-
 .../tasks/JobCheckpointingSettingsTest.java     |   3 +-
 .../jobmanager/JobManagerHARecoveryTest.java    |   4 +-
 .../runtime/jobmanager/JobManagerTest.java      |  16 +-
 .../flink/runtime/jobmanager/JobSubmitTest.java |   5 +-
 ...obCancellationWithSavepointHandlersTest.java |  10 +-
 .../CheckpointConfigHandlerTest.java            |  31 +-
 .../CheckpointStatsDetailsHandlerTest.java      |   8 +-
 .../checkpoints/CheckpointStatsHandlerTest.java |  11 +-
 .../runtime/state/EmptyStreamStateHandle.java   |  45 +++
 .../AbstractFileStateBackendTest.java           | 309 +++++++++++++++++
 .../FixFileFsStateOutputStreamTest.java         | 285 ++++++++++++++++
 .../memory/MemoryCheckpointStorageTest.java     |  26 ++
 .../runtime/jobmanager/JobManagerITCase.scala   |  12 +-
 .../testingUtils/TestingJobManagerLike.scala    |  29 +-
 .../api/graph/StreamingJobGraphGenerator.java   |  12 +-
 .../tasks/StreamTaskTerminationTest.java        |  18 +-
 .../tasks/TaskCheckpointingBehaviourTest.java   |   2 +-
 .../test/checkpointing/SavepointITCase.java     |   9 -
 .../utils/SavepointMigrationTestBase.java       |   2 -
 .../streaming/runtime/StateBackendITCase.java   |  21 +-
 80 files changed, 3167 insertions(+), 2064 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 6bcd595..072f3a7 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
@@ -28,12 +28,14 @@ import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.util.AbstractID;
 
@@ -41,7 +43,6 @@ import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.NativeLibraryLoader;
 import org.rocksdb.RocksDB;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -300,7 +301,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu
 
 	private void lazyInitializeForJob(
 			Environment env,
-			String operatorIdentifier) throws IOException {
+			@SuppressWarnings("unused") String operatorIdentifier) throws IOException {
 
 		if (isInitialized) {
 			return;
@@ -352,9 +353,22 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu
 		return initializedDbBasePaths[ni];
 	}
 
+	// ------------------------------------------------------------------------
+	//  Checkpoint initialization and persistent storage
+	// ------------------------------------------------------------------------
+
+	@Override
+	public StreamStateHandle resolveCheckpoint(String pointer) throws IOException {
+		return checkpointStreamBackend.resolveCheckpoint(pointer);
+	}
+
 	@Override
-	public CheckpointStreamFactory createStreamFactory(JobID jobId,
-			String operatorIdentifier) throws IOException {
+	public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+		return checkpointStreamBackend.createCheckpointStorage(jobId);
+	}
+
+	@Override
+	public CheckpointStreamFactory createStreamFactory(JobID jobId, String operatorIdentifier) throws IOException {
 		return checkpointStreamBackend.createStreamFactory(jobId, operatorIdentifier);
 	}
 
@@ -367,6 +381,10 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu
 		return checkpointStreamBackend.createSavepointStreamFactory(jobId, operatorIdentifier, targetLocation);
 	}
 
+	// ------------------------------------------------------------------------
+	//  State holding data structures
+	// ------------------------------------------------------------------------
+
 	@Override
 	public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 			Environment env,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 c79e3d7..6770ec3 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
@@ -23,13 +23,13 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.metrics.jmx.JMXReporter;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
@@ -56,10 +56,9 @@ import static org.junit.Assert.assertEquals;
  * Tests to verify JMX reporter functionality on the JobManager.
  */
 public class JMXJobManagerMetricTest {
+
 	/**
 	 * Tests that metrics registered on the JobManager are actually accessible via JMX.
-	 *
-	 * @throws Exception
 	 */
 	@Test
 	public void testJobManagerJMXMetricAccess() throws Exception {
@@ -89,7 +88,7 @@ public class JMXJobManagerMetricTest {
 					500,
 					50,
 					5,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					true),
 				null));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 40fa2bd..95ca5d7 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
@@ -22,10 +22,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.runtime.checkpoint.hooks.MasterHooks;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointLoader;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.Execution;
@@ -36,12 +33,14 @@ import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.SharedStateRegistryFactory;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StringUtils;
@@ -119,12 +118,7 @@ public class CheckpointCoordinator {
 
 	/** The root checkpoint state backend, which is responsible for initializing the
 	 * checkpoint, storing the metadata, and cleaning up the checkpoint */
-	private final StateBackend checkpointStateBackend;
-
-	/** 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;
+	private final CheckpointStorage checkpointStorage;
 
 	/** A list of recent checkpoint IDs, to identify late messages (vs invalid ones) */
 	private final ArrayDeque<Long> recentPendingCheckpoints;
@@ -194,29 +188,23 @@ public class CheckpointCoordinator {
 			long checkpointTimeout,
 			long minPauseBetweenCheckpoints,
 			int maxConcurrentCheckpointAttempts,
-			ExternalizedCheckpointSettings externalizeSettings,
+			CheckpointRetentionPolicy retentionPolicy,
 			ExecutionVertex[] tasksToTrigger,
 			ExecutionVertex[] tasksToWaitFor,
 			ExecutionVertex[] tasksToCommitTo,
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore completedCheckpointStore,
-			@Nullable String checkpointDirectory,
 			StateBackend checkpointStateBackend,
 			Executor executor,
 			SharedStateRegistryFactory sharedStateRegistryFactory) {
 
 		// sanity checks
+		checkNotNull(checkpointStateBackend);
 		checkArgument(baseInterval > 0, "Checkpoint timeout must be larger than zero");
 		checkArgument(checkpointTimeout >= 1, "Checkpoint timeout must be larger than zero");
 		checkArgument(minPauseBetweenCheckpoints >= 0, "minPauseBetweenCheckpoints must be >= 0");
 		checkArgument(maxConcurrentCheckpointAttempts >= 1, "maxConcurrentCheckpointAttempts must be >= 1");
 
-		if (externalizeSettings.externalizeCheckpoints() && checkpointDirectory == null) {
-			throw new IllegalStateException("CheckpointConfig says to persist periodic " +
-					"checkpoints, but no checkpoint directory has been configured. You can " +
-					"configure configure one via key '" + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key() + "'.");
-		}
-
 		// max "in between duration" can be one year - this is to prevent numeric overflows
 		if (minPauseBetweenCheckpoints > 365L * 24 * 60 * 60 * 1_000) {
 			minPauseBetweenCheckpoints = 365L * 24 * 60 * 60 * 1_000;
@@ -239,8 +227,6 @@ public class CheckpointCoordinator {
 		this.pendingCheckpoints = new LinkedHashMap<>();
 		this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
 		this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
-		this.checkpointStateBackend = checkNotNull(checkpointStateBackend);
-		this.checkpointDirectory = checkpointDirectory;
 		this.executor = checkNotNull(executor);
 		this.sharedStateRegistryFactory = checkNotNull(sharedStateRegistryFactory);
 		this.sharedStateRegistry = sharedStateRegistryFactory.create(executor);
@@ -256,14 +242,11 @@ public class CheckpointCoordinator {
 		this.timer.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
 		this.timer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
 
-		if (externalizeSettings.externalizeCheckpoints()) {
-			LOG.info("Persisting periodic checkpoints externally at {}.", checkpointDirectory);
-			checkpointProperties = CheckpointProperties.forExternalizedCheckpoint(externalizeSettings.deleteOnCancellation());
-		} else {
-			checkpointProperties = CheckpointProperties.forStandardCheckpoint();
-		}
+		this.checkpointProperties = CheckpointProperties.forCheckpoint(retentionPolicy);
 
 		try {
+			this.checkpointStorage = checkpointStateBackend.createCheckpointStorage(job);
+
 			// Make sure the checkpoint ID enumerator is running. Possibly
 			// issues a blocking call to ZooKeeper.
 			checkpointIDCounter.start();
@@ -280,7 +263,7 @@ public class CheckpointCoordinator {
 	 * Adds the given master hook to the checkpoint coordinator. This method does nothing, if
 	 * the checkpoint coordinator already contained a hook with the same ID (as defined via
 	 * {@link MasterTriggerRestoreHook#getIdentifier()}).
-	 * 
+	 *
 	 * @param hook The hook to add.
 	 * @return True, if the hook was added, false if the checkpoint coordinator already
 	 *         contained a hook with the same ID.
@@ -366,52 +349,32 @@ public class CheckpointCoordinator {
 	 * Triggers a savepoint with the given savepoint directory as a target.
 	 *
 	 * @param timestamp The timestamp for the savepoint.
-	 * @param targetDirectory Target directory for the savepoint.
+	 * @param targetLocation Target location for the savepoint, optional. If null, the
+	 *                       state backend's configured default will be used.
 	 * @return A future to the completed checkpoint
 	 * @throws IllegalStateException If no savepoint directory has been
 	 *                               specified and no default savepoint directory has been
 	 *                               configured
 	 * @throws Exception             Failures during triggering are forwarded
 	 */
-	public CompletableFuture<CompletedCheckpoint> triggerSavepoint(long timestamp, String targetDirectory) throws Exception {
-		checkNotNull(targetDirectory, "Savepoint target directory");
-
-		CheckpointProperties props = CheckpointProperties.forStandardSavepoint();
+	public CompletableFuture<CompletedCheckpoint> triggerSavepoint(
+			long timestamp,
+			@Nullable String targetLocation) throws Exception {
 
-		// Create the unique savepoint directory
-		final String savepointDirectory = SavepointStore
-			.createSavepointDirectory(targetDirectory, job);
+		CheckpointProperties props = CheckpointProperties.forSavepoint();
 
 		CheckpointTriggerResult triggerResult = triggerCheckpoint(
 			timestamp,
 			props,
-			savepointDirectory,
+			targetLocation,
 			false);
 
-		CompletableFuture<CompletedCheckpoint> result;
-
 		if (triggerResult.isSuccess()) {
-			result = triggerResult.getPendingCheckpoint().getCompletionFuture();
+			return triggerResult.getPendingCheckpoint().getCompletionFuture();
 		} else {
 			Throwable cause = new Exception("Failed to trigger savepoint: " + triggerResult.getFailureReason().message());
 			return FutureUtils.completedExceptionally(cause);
 		}
-
-		// Make sure to remove the created base directory on Exceptions
-		result.whenCompleteAsync(
-			(CompletedCheckpoint checkpoint, Throwable throwable) -> {
-				if (throwable != null) {
-					try {
-						SavepointStore.deleteSavepointDirectory(savepointDirectory);
-					} catch (Throwable t) {
-						LOG.warn("Failed to delete savepoint directory " + savepointDirectory
-							+ " after failed savepoint.", t);
-					}
-				}
-			},
-			executor);
-
-		return result;
 	}
 
 	/**
@@ -425,7 +388,7 @@ public class CheckpointCoordinator {
 	 * @return <code>true</code> if triggering the checkpoint succeeded.
 	 */
 	public boolean triggerCheckpoint(long timestamp, boolean isPeriodic) {
-		return triggerCheckpoint(timestamp, checkpointProperties, checkpointDirectory, isPeriodic).isSuccess();
+		return triggerCheckpoint(timestamp, checkpointProperties, null, isPeriodic).isSuccess();
 	}
 
 	/**
@@ -444,7 +407,7 @@ public class CheckpointCoordinator {
 
 			case CHECKPOINT:
 				CheckpointTriggerResult triggerResult =
-					triggerCheckpoint(timestamp, checkpointProperties, checkpointDirectory, false);
+					triggerCheckpoint(timestamp, checkpointProperties, null, false);
 
 				if (triggerResult.isSuccess()) {
 					return triggerResult.getPendingCheckpoint().getCompletionFuture();
@@ -462,14 +425,9 @@ public class CheckpointCoordinator {
 	CheckpointTriggerResult triggerCheckpoint(
 			long timestamp,
 			CheckpointProperties props,
-			String targetDirectory,
+			@Nullable String externalSavepointLocation,
 			boolean isPeriodic) {
 
-		// Sanity check
-		if (props.externalizeCheckpoint() && targetDirectory == null) {
-			throw new IllegalStateException("No target directory specified to persist checkpoint to.");
-		}
-
 		// make some eager pre-checks
 		synchronized (lock) {
 			// abort if the coordinator has been shutdown in the meantime
@@ -557,11 +515,18 @@ public class CheckpointCoordinator {
 		// may issue blocking operations. Using a different lock than the coordinator-wide lock,
 		// we avoid blocking the processing of 'acknowledge/decline' messages during that time.
 		synchronized (triggerLock) {
+
+			final CheckpointStorageLocation checkpointStorageLocation;
 			final long checkpointID;
+
 			try {
 				// this must happen outside the coordinator-wide lock, because it communicates
 				// with external services (in HA mode) and may block for a while.
 				checkpointID = checkpointIdCounter.getAndIncrement();
+
+				checkpointStorageLocation = props.isSavepoint() ?
+						checkpointStorage.initializeLocationForSavepoint(checkpointID, externalSavepointLocation) :
+						checkpointStorage.initializeLocationForCheckpoint(checkpointID);
 			}
 			catch (Throwable t) {
 				int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet();
@@ -575,7 +540,7 @@ public class CheckpointCoordinator {
 				timestamp,
 				ackTasks,
 				props,
-				targetDirectory,
+				checkpointStorageLocation,
 				executor);
 
 			if (statsTracker != null) {
@@ -588,21 +553,18 @@ public class CheckpointCoordinator {
 			}
 
 			// schedule the timer that will clean up the expired checkpoints
-			final Runnable canceller = new Runnable() {
-				@Override
-				public void run() {
-					synchronized (lock) {
-						// only do the work if the checkpoint is not discarded anyways
-						// note that checkpoint completion discards the pending checkpoint object
-						if (!checkpoint.isDiscarded()) {
-							LOG.info("Checkpoint " + checkpointID + " expired before completing.");
+			final Runnable canceller = () -> {
+				synchronized (lock) {
+					// only do the work if the checkpoint is not discarded anyways
+					// note that checkpoint completion discards the pending checkpoint object
+					if (!checkpoint.isDiscarded()) {
+						LOG.info("Checkpoint " + checkpointID + " expired before completing.");
 
-							checkpoint.abortExpired();
-							pendingCheckpoints.remove(checkpointID);
-							rememberRecentCheckpointId(checkpointID);
+						checkpoint.abortExpired();
+						pendingCheckpoints.remove(checkpointID);
+						rememberRecentCheckpointId(checkpointID);
 
-							triggerQueuedRequests();
-						}
+						triggerQueuedRequests();
 					}
 				}
 			};
@@ -675,7 +637,7 @@ public class CheckpointCoordinator {
 				if (!props.isSavepoint()) {
 					checkpointOptions = CheckpointOptions.forCheckpoint();
 				} else {
-					checkpointOptions = CheckpointOptions.forSavepoint(targetDirectory);
+					checkpointOptions = CheckpointOptions.forSavepoint(checkpointStorageLocation.getLocationAsPointer());
 				}
 
 				// send the messages to the tasks that trigger their checkpoint
@@ -699,6 +661,14 @@ public class CheckpointCoordinator {
 				if (!checkpoint.isDiscarded()) {
 					checkpoint.abortError(new Exception("Failed to trigger checkpoint", t));
 				}
+
+				try {
+					checkpointStorageLocation.disposeOnFailure();
+				}
+				catch (Throwable t2) {
+					LOG.warn("Cannot dispose failed checkpoint storage location {}", checkpointStorageLocation, t2);
+				}
+
 				return new CheckpointTriggerResult(CheckpointDeclineReason.EXCEPTION);
 			}
 
@@ -718,7 +688,7 @@ public class CheckpointCoordinator {
 			throw new IllegalArgumentException("Received DeclineCheckpoint message for job " +
 				message.getJob() + " while this coordinator handles job " + job);
 		}
-		
+
 		final long checkpointId = message.getCheckpointId();
 		final String reason = (message.getReason() != null ? message.getReason().getMessage() : "");
 
@@ -778,7 +748,7 @@ public class CheckpointCoordinator {
 		}
 
 		final long checkpointId = message.getCheckpointId();
-		
+
 		synchronized (lock) {
 			// we need to check inside the lock for being shutdown as well, otherwise we
 			// get races and invalid error log messages
@@ -854,7 +824,7 @@ public class CheckpointCoordinator {
 	/**
 	 * Try to complete the given pending checkpoint.
 	 *
-	 * Important: This method should only be called in the checkpoint lock scope.
+	 * <p>Important: This method should only be called in the checkpoint lock scope.
 	 *
 	 * @param pendingCheckpoint to complete
 	 * @throws CheckpointException if the completion failed
@@ -869,13 +839,9 @@ public class CheckpointCoordinator {
 
 		try {
 			try {
-				// externalize the checkpoint if required
-				if (pendingCheckpoint.getProps().externalizeCheckpoint()) {
-					completedCheckpoint = pendingCheckpoint.finalizeCheckpointExternalized();
-				} else {
-					completedCheckpoint = pendingCheckpoint.finalizeCheckpointNonExternalized();
-				}
-			} catch (Exception e1) {
+				completedCheckpoint = pendingCheckpoint.finalizeCheckpoint();
+			}
+			catch (Exception e1) {
 				// abort the current pending checkpoint if we fails to finalize the pending checkpoint.
 				if (!pendingCheckpoint.isDiscarded()) {
 					pendingCheckpoint.abortError(e1);
@@ -1122,38 +1088,40 @@ public class CheckpointCoordinator {
 	}
 
 	/**
-	 * Restore the state with given savepoint
-	 * 
-	 * @param savepointPath    Location of the savepoint
-	 * @param allowNonRestored True if allowing checkpoint state that cannot be 
+	 * Restore the state with given savepoint.
+	 *
+	 * @param savepointPointer The pointer to the savepoint.
+	 * @param allowNonRestored True if allowing checkpoint state that cannot be
 	 *                         mapped to any job vertex in tasks.
-	 * @param tasks            Map of job vertices to restore. State for these 
-	 *                         vertices is restored via 
+	 * @param tasks            Map of job vertices to restore. State for these
+	 *                         vertices is restored via
 	 *                         {@link Execution#setInitialState(TaskStateSnapshot)}.
-	 * @param userClassLoader  The class loader to resolve serialized classes in 
-	 *                         legacy savepoint versions. 
+	 * @param userClassLoader  The class loader to resolve serialized classes in
+	 *                         legacy savepoint versions.
 	 */
 	public boolean restoreSavepoint(
-			String savepointPath, 
+			String savepointPointer,
 			boolean allowNonRestored,
 			Map<JobVertexID, ExecutionJobVertex> tasks,
 			ClassLoader userClassLoader) throws Exception {
-		
-		Preconditions.checkNotNull(savepointPath, "The savepoint path cannot be null.");
-		
-		LOG.info("Starting job from savepoint {} ({})", 
-				savepointPath, (allowNonRestored ? "allowing non restored state" : ""));
+
+		Preconditions.checkNotNull(savepointPointer, "The savepoint path cannot be null.");
+
+		LOG.info("Starting job from savepoint {} ({})",
+				savepointPointer, (allowNonRestored ? "allowing non restored state" : ""));
+
+		final StreamStateHandle metadataHandle = checkpointStorage.resolveCheckpoint(savepointPointer);
 
 		// Load the savepoint as a checkpoint into the system
-		CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint(
-				job, tasks, savepointPath, userClassLoader, allowNonRestored);
+		CompletedCheckpoint savepoint = Checkpoints.loadAndValidateCheckpoint(
+				job, tasks, savepointPointer, metadataHandle, userClassLoader, allowNonRestored);
 
 		completedCheckpointStore.addCheckpoint(savepoint);
-		
+
 		// Reset the checkpoint ID counter
 		long nextCheckpointId = savepoint.getCheckpointID() + 1;
 		checkpointIdCounter.setCount(nextCheckpointId);
-		
+
 		LOG.info("Reset the checkpoint ID to {}.", nextCheckpointId);
 
 		return restoreLatestCheckpointedState(tasks, true, allowNonRestored);
@@ -1185,6 +1153,10 @@ public class CheckpointCoordinator {
 		}
 	}
 
+	public CheckpointStorage getCheckpointStorage() {
+		return checkpointStorage;
+	}
+
 	public CompletedCheckpointStore getCheckpointStore() {
 		return completedCheckpointStore;
 	}
@@ -1221,7 +1193,7 @@ public class CheckpointCoordinator {
 
 			periodicScheduling = true;
 			currentPeriodicTrigger = timer.scheduleAtFixedRate(
-					new ScheduledTrigger(), 
+					new ScheduledTrigger(),
 					baseInterval, baseInterval, TimeUnit.MILLISECONDS);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java
index 1233b6e..8d6346c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java
@@ -34,11 +34,10 @@ import java.io.Serializable;
  */
 public class CheckpointProperties implements Serializable {
 
-	private static final long serialVersionUID = -8835900655844879469L;
+	private static final long serialVersionUID = -8835900655844879470L;
 
 	private final boolean forced;
 
-	private final boolean externalize;
 	private final boolean savepoint;
 
 	private final boolean discardSubsumed;
@@ -49,7 +48,6 @@ public class CheckpointProperties implements Serializable {
 
 	CheckpointProperties(
 			boolean forced,
-			boolean externalize,
 			boolean savepoint,
 			boolean discardSubsumed,
 			boolean discardFinished,
@@ -58,20 +56,12 @@ public class CheckpointProperties implements Serializable {
 			boolean discardSuspended) {
 
 		this.forced = forced;
-		this.externalize = externalize;
 		this.savepoint = savepoint;
 		this.discardSubsumed = discardSubsumed;
 		this.discardFinished = discardFinished;
 		this.discardCancelled = discardCancelled;
 		this.discardFailed = discardFailed;
 		this.discardSuspended = discardSuspended;
-
-		// Not persisted, but needs manual clean up
-		if (!externalize && !(discardSubsumed && discardFinished && discardCancelled
-				&& discardFailed && discardSuspended)) {
-			throw new IllegalStateException("CheckpointProperties say to *not* persist the " +
-					"checkpoint, but the checkpoint requires manual cleanup.");
-		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -93,18 +83,6 @@ public class CheckpointProperties implements Serializable {
 		return forced;
 	}
 
-	/**
-	 * Returns whether the checkpoint should be persisted externally.
-	 *
-	 * @return <code>true</code> if the checkpoint should be persisted
-	 * externally; <code>false</code> otherwise.
-	 *
-	 * @see PendingCheckpoint
-	 */
-	boolean externalizeCheckpoint() {
-		return externalize;
-	}
-
 	// ------------------------------------------------------------------------
 	// Garbage collection behaviour
 	// ------------------------------------------------------------------------
@@ -203,7 +181,6 @@ public class CheckpointProperties implements Serializable {
 
 		CheckpointProperties that = (CheckpointProperties) o;
 		return forced == that.forced &&
-				externalize == that.externalize &&
 				savepoint == that.savepoint &&
 				discardSubsumed == that.discardSubsumed &&
 				discardFinished == that.discardFinished &&
@@ -215,7 +192,6 @@ public class CheckpointProperties implements Serializable {
 	@Override
 	public int hashCode() {
 		int result = (forced ? 1 : 0);
-		result = 31 * result + (externalize ? 1 : 0);
 		result = 31 * result + (savepoint ? 1 : 0);
 		result = 31 * result + (discardSubsumed ? 1 : 0);
 		result = 31 * result + (discardFinished ? 1 : 0);
@@ -229,7 +205,6 @@ public class CheckpointProperties implements Serializable {
 	public String toString() {
 		return "CheckpointProperties{" +
 				"forced=" + forced +
-				", externalized=" + externalizeCheckpoint() +
 				", savepoint=" + savepoint +
 				", discardSubsumed=" + discardSubsumed +
 				", discardFinished=" + discardFinished +
@@ -241,8 +216,7 @@ public class CheckpointProperties implements Serializable {
 
 	// ------------------------------------------------------------------------
 
-	private static final CheckpointProperties STANDARD_SAVEPOINT = new CheckpointProperties(
-			true,
+	private static final CheckpointProperties SAVEPOINT = new CheckpointProperties(
 			true,
 			true,
 			false,
@@ -251,77 +225,66 @@ public class CheckpointProperties implements Serializable {
 			false,
 			false);
 
-	private static final CheckpointProperties STANDARD_CHECKPOINT = new CheckpointProperties(
-			false,
+	private static final CheckpointProperties CHECKPOINT_NEVER_RETAINED = new CheckpointProperties(
 			false,
 			false,
 			true,
-			true,
-			true,
-			true,
-			true);
+			true,  // Delete on success
+			true,  // Delete on cancellation
+			true,  // Delete on failure
+			true); // Delete on suspension
 
-	private static final CheckpointProperties EXTERNALIZED_CHECKPOINT_RETAINED = new CheckpointProperties(
+	private static final CheckpointProperties CHECKPOINT_RETAINED_ON_FAILURE = new CheckpointProperties(
 			false,
-			true,
 			false,
 			true,
-			true,
-			false, // Retain on cancellation
-			false,
-			false); // Retain on suspension
+			true,  // Delete on success
+			true,  // Delete on cancellation
+			false, // Retain on failure
+			true); // Delete on suspension
 
-	private static final CheckpointProperties EXTERNALIZED_CHECKPOINT_DELETED = new CheckpointProperties(
+	private static final CheckpointProperties CHECKPOINT_RETAINED_ON_CANCELLATION = new CheckpointProperties(
 			false,
-			true,
 			false,
 			true,
-			true,
-			true, // Delete on cancellation
-			false,
-			true); // Delete on suspension
+			true,   // Delete on success
+			false,  // Retain on cancellation
+			false,  // Retain on failure
+			false); // Retain on suspension
+
 
 	/**
 	 * Creates the checkpoint properties for a (manually triggered) savepoint.
 	 *
-	 * <p>Savepoints are forced and persisted externally. They have to be
+	 * <p>Savepoints are not queued due to time trigger limits. They have to be
 	 * garbage collected manually.
 	 *
 	 * @return Checkpoint properties for a (manually triggered) savepoint.
 	 */
-	public static CheckpointProperties forStandardSavepoint() {
-		return STANDARD_SAVEPOINT;
-	}
-
-	/**
-	 * Creates the checkpoint properties for a regular checkpoint.
-	 *
-	 * <p>Regular checkpoints are not forced and not persisted externally. They
-	 * are garbage collected automatically.
-	 *
-	 * @return Checkpoint properties for a regular checkpoint.
-	 */
-	public static CheckpointProperties forStandardCheckpoint() {
-		return STANDARD_CHECKPOINT;
+	public static CheckpointProperties forSavepoint() {
+		return SAVEPOINT;
 	}
 
 	/**
-	 * Creates the checkpoint properties for an external checkpoint.
+	 * Creates the checkpoint properties for a checkpoint.
 	 *
-	 * <p>External checkpoints are not forced, but persisted externally. They
-	 * are garbage collected automatically, except when the owning job
+	 * <p>Checkpoints may be queued in case too many other checkpoints are currently happening.
+	 * They are garbage collected automatically, except when the owning job
 	 * terminates in state {@link JobStatus#FAILED}. The user is required to
 	 * configure the clean up behaviour on job cancellation.
 	 *
-	 * @param deleteOnCancellation Flag indicating whether to discard on cancellation.
-	 *
 	 * @return Checkpoint properties for an external checkpoint.
 	 */
-	public static CheckpointProperties forExternalizedCheckpoint(boolean deleteOnCancellation) {
-		if (deleteOnCancellation) {
-			return EXTERNALIZED_CHECKPOINT_DELETED;
-		} else {
-			return EXTERNALIZED_CHECKPOINT_RETAINED;
+	public static CheckpointProperties forCheckpoint(CheckpointRetentionPolicy policy) {
+		switch (policy) {
+			case NEVER_RETAIN_AFTER_TERMINATION:
+				return CHECKPOINT_NEVER_RETAINED;
+			case RETAIN_ON_FAILURE:
+				return CHECKPOINT_RETAINED_ON_FAILURE;
+			case RETAIN_ON_CANCELLATION:
+				return CHECKPOINT_RETAINED_ON_CANCELLATION;
+			default:
+				throw new IllegalArgumentException("unknown policy: " + policy);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRetentionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRetentionPolicy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRetentionPolicy.java
new file mode 100644
index 0000000..3bd124d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRetentionPolicy.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.flink.annotation.Internal;
+
+/**
+ * Policy for whether checkpoints are retained after a job terminates.
+ */
+@Internal
+public enum CheckpointRetentionPolicy {
+
+	/** Checkpoints should be retained on cancellation and failure. */
+	RETAIN_ON_CANCELLATION,
+
+	/** Checkpoints should be retained on failure, but not on cancellation. */
+	RETAIN_ON_FAILURE,
+
+	/** Checkpoints should always be cleaned up when an application reaches a terminal state. */
+	NEVER_RETAIN_AFTER_TERMINATION;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java
new file mode 100644
index 0000000..bfa7d45
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java
@@ -0,0 +1,327 @@
+/*
+ * 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 org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.checkpoint.savepoint.Savepoint;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializer;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.AbstractFileStateBackend;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A utility class with the methods to write/load/dispose the checkpoint and savepoint metadata.
+ *
+ * <p>Stored checkpoint metadata files have the following format:
+ * <pre>[MagicNumber (int) | Format Version (int) | Checkpoint Metadata (variable)]</pre>
+ *
+ * <p>The actual savepoint serialization is version-specific via the {@link SavepointSerializer}.
+ */
+public class Checkpoints {
+
+	private static final Logger LOG = LoggerFactory.getLogger(Checkpoints.class);
+
+	/** Magic number at the beginning of every checkpoint metadata file, for sanity checks. */
+	public static final int HEADER_MAGIC_NUMBER = 0x4960672d;
+
+	// ------------------------------------------------------------------------
+	//  Writing out checkpoint metadata
+	// ------------------------------------------------------------------------
+
+	public static <T extends Savepoint> void storeCheckpointMetadata(
+			T checkpointMetadata,
+			OutputStream out) throws IOException {
+
+		DataOutputStream dos = new DataOutputStream(out);
+		storeCheckpointMetadata(checkpointMetadata, dos);
+	}
+
+	public static <T extends Savepoint> void storeCheckpointMetadata(
+			T checkpointMetadata,
+			DataOutputStream out) throws IOException {
+
+		// write generic header
+		out.writeInt(HEADER_MAGIC_NUMBER);
+		out.writeInt(checkpointMetadata.getVersion());
+
+		// write checkpoint metadata
+		SavepointSerializer<T> serializer = SavepointSerializers.getSerializer(checkpointMetadata);
+		serializer.serialize(checkpointMetadata, out);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reading and validating checkpoint metadata
+	// ------------------------------------------------------------------------
+
+	public static Savepoint loadCheckpointMetadata(DataInputStream in, ClassLoader classLoader) throws IOException {
+		checkNotNull(in, "input stream");
+		checkNotNull(classLoader, "classLoader");
+
+		final int magicNumber = in.readInt();
+
+		if (magicNumber == HEADER_MAGIC_NUMBER) {
+			final int version = in.readInt();
+			final SavepointSerializer<?> serializer = SavepointSerializers.getSerializer(version);
+
+			if (serializer != null) {
+				return serializer.deserialize(in, classLoader);
+			}
+			else {
+				throw new IOException("Unrecognized checkpoint version number: " + version);
+			}
+		}
+		else {
+			throw new IOException("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.");
+		}
+	}
+
+	@SuppressWarnings("deprecation")
+	public static CompletedCheckpoint loadAndValidateCheckpoint(
+			JobID jobId,
+			Map<JobVertexID, ExecutionJobVertex> tasks,
+			String checkpointPointer,
+			StreamStateHandle metadataHandle,
+			ClassLoader classLoader,
+			boolean allowNonRestoredState) throws IOException {
+
+		checkNotNull(jobId, "jobId");
+		checkNotNull(tasks, "tasks");
+		checkNotNull(checkpointPointer, "checkpointPointer");
+		checkNotNull(metadataHandle, "metadataHandle");
+		checkNotNull(classLoader, "classLoader");
+
+		// (1) load the savepoint
+		final Savepoint rawCheckpointMetadata;
+		try (FSDataInputStream in = metadataHandle.openInputStream()) {
+			DataInputStream dis = new DataInputStream(in);
+			rawCheckpointMetadata = loadCheckpointMetadata(dis, classLoader);
+		}
+
+		final Savepoint checkpointMetadata = rawCheckpointMetadata.getTaskStates() == null ?
+				rawCheckpointMetadata :
+				SavepointV2.convertToOperatorStateSavepointV2(tasks, rawCheckpointMetadata);
+
+		// generate mapping from operator to task
+		Map<OperatorID, ExecutionJobVertex> operatorToJobVertexMapping = new HashMap<>();
+		for (ExecutionJobVertex task : tasks.values()) {
+			for (OperatorID operatorID : task.getOperatorIDs()) {
+				operatorToJobVertexMapping.put(operatorID, task);
+			}
+		}
+
+		// (2) validate it (parallelism, etc)
+		boolean expandedToLegacyIds = false;
+
+		HashMap<OperatorID, OperatorState> operatorStates = new HashMap<>(checkpointMetadata.getOperatorStates().size());
+		for (OperatorState operatorState : checkpointMetadata.getOperatorStates()) {
+
+			ExecutionJobVertex executionJobVertex = operatorToJobVertexMapping.get(operatorState.getOperatorID());
+
+			// on the first time we can not find the execution job vertex for an id, we also consider alternative ids,
+			// for example as generated from older flink versions, to provide backwards compatibility.
+			if (executionJobVertex == null && !expandedToLegacyIds) {
+				operatorToJobVertexMapping = ExecutionJobVertex.includeAlternativeOperatorIDs(operatorToJobVertexMapping);
+				executionJobVertex = operatorToJobVertexMapping.get(operatorState.getOperatorID());
+				expandedToLegacyIds = true;
+				LOG.info("Could not find ExecutionJobVertex. Including user-defined OperatorIDs in search.");
+			}
+
+			if (executionJobVertex != null) {
+
+				if (executionJobVertex.getMaxParallelism() == operatorState.getMaxParallelism()
+						|| !executionJobVertex.isMaxParallelismConfigured()) {
+					operatorStates.put(operatorState.getOperatorID(), operatorState);
+				} else {
+					String msg = String.format("Failed to rollback to checkpoint/savepoint %s. " +
+									"Max parallelism mismatch between checkpoint/savepoint state and new program. " +
+									"Cannot map operator %s with max parallelism %d to new program with " +
+									"max parallelism %d. This indicates that the program has been changed " +
+									"in a non-compatible way after the checkpoint/savepoint.",
+							checkpointMetadata,
+							operatorState.getOperatorID(),
+							operatorState.getMaxParallelism(),
+							executionJobVertex.getMaxParallelism());
+
+					throw new IllegalStateException(msg);
+				}
+			} else if (allowNonRestoredState) {
+				LOG.info("Skipping savepoint state for operator {}.", operatorState.getOperatorID());
+			} else {
+				for (OperatorSubtaskState operatorSubtaskState : operatorState.getStates()) {
+					if (operatorSubtaskState.hasState()) {
+						String msg = String.format("Failed to rollback to checkpoint/savepoint %s. " +
+										"Cannot map checkpoint/savepoint state for operator %s to the new program, " +
+										"because the operator is not available in the new program. If " +
+										"you want to allow to skip this, you can set the --allowNonRestoredState " +
+										"option on the CLI.",
+								checkpointPointer, operatorState.getOperatorID());
+
+						throw new IllegalStateException(msg);
+					}
+				}
+
+				LOG.info("Skipping empty savepoint state for operator {}.", operatorState.getOperatorID());
+			}
+		}
+
+		// (3) convert to checkpoint so the system can fall back to it
+		CheckpointProperties props = CheckpointProperties.forSavepoint();
+
+		return new CompletedCheckpoint(
+				jobId,
+				checkpointMetadata.getCheckpointId(),
+				0L,
+				0L,
+				operatorStates,
+				checkpointMetadata.getMasterStates(),
+				props,
+				metadataHandle,
+				checkpointPointer);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Savepoint Disposal Hooks
+	// ------------------------------------------------------------------------
+
+	public static void disposeSavepoint(
+			String pointer,
+			StateBackend stateBackend,
+			ClassLoader classLoader) throws IOException, FlinkException {
+
+		checkNotNull(pointer, "location");
+		checkNotNull(stateBackend, "stateBackend");
+		checkNotNull(classLoader, "classLoader");
+
+		final StreamStateHandle metadataHandle = stateBackend.resolveCheckpoint(pointer);
+
+		// load the savepoint object (the metadata) to have all the state handles that we need
+		// to dispose of all state
+		final Savepoint savepoint;
+		try (FSDataInputStream in = metadataHandle.openInputStream();
+			DataInputStream dis = new DataInputStream(in)) {
+
+				savepoint = loadCheckpointMetadata(dis, classLoader);
+		}
+
+		Exception exception = null;
+
+		// first dispose the savepoint metadata, so that the savepoint is not
+		// addressable any more even if the following disposal fails
+		try {
+			metadataHandle.discardState();
+		}
+		catch (Exception e) {
+			exception = e;
+		}
+
+		// now dispose the savepoint data
+		try {
+			savepoint.dispose();
+		}
+		catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		// until we have the proper hooks to delete full directories via the checkpoint storage,
+		// we need to have a special case here to remove the empty directory
+		if (stateBackend instanceof AbstractFileStateBackend && metadataHandle instanceof FileStateHandle) {
+			Path dir = ((FileStateHandle) metadataHandle).getFilePath().getParent();
+			FileUtils.deletePathIfEmpty(dir.getFileSystem(), dir);
+		}
+
+		if (exception != null) {
+			ExceptionUtils.rethrowIOException(exception);
+		}
+	}
+
+	public static void disposeSavepoint(
+			String pointer,
+			Configuration configuration,
+			ClassLoader classLoader,
+			@Nullable Logger logger) throws IOException, FlinkException {
+
+		checkNotNull(pointer, "location");
+		checkNotNull(configuration, "configuration");
+		checkNotNull(classLoader, "classLoader");
+
+		if (logger != null) {
+			logger.info("Attempting to load configured state backend for savepoint disposal");
+		}
+
+		StateBackend backend = null;
+		try {
+			backend = StateBackendLoader.loadStateBackendFromConfig(configuration, classLoader, null);
+
+			if (backend == null && logger != null) {
+				logger.info("No state backend configured, attempting to dispose savepoint " +
+						"with default backend (file system based)");
+			}
+		}
+		catch (Throwable t) {
+			// catches exceptions and errors (like linking errors)
+			if (logger != null) {
+				logger.info("Could not load configured state backend.");
+				logger.debug("Detailed exception:", t);
+			}
+		}
+
+		if (backend == null) {
+			// We use the memory state backend by default. The MemoryStateBackend is actually
+			// FileSystem-based for metadata
+			backend = new MemoryStateBackend();
+		}
+
+		disposeSavepoint(pointer, backend, classLoader);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class contains only static utility methods and is not meant to be instantiated. */
+	private Checkpoints() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 d6d0827..801232c 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
@@ -25,10 +25,12 @@ import org.apache.flink.runtime.state.SharedStateRegistry;
 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;
 
 import javax.annotation.Nullable;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -44,32 +46,22 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * 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
+ *
+ * <p>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
+ *
+ * <h2>Metadata Persistence</h2>
+ *
+ * <p>The metadata of the CompletedCheckpoint is also persisted in an external storage
+ * system. Checkpoints have an external pointer, which points to the metadata. For example
+ * when storing a checkpoint in 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 {
 
@@ -79,10 +71,10 @@ public class CompletedCheckpoint implements Serializable {
 
 	// ------------------------------------------------------------------------
 
-	/** The ID of the job that the checkpoint belongs to */
+	/** The ID of the job that the checkpoint belongs to. */
 	private final JobID job;
 
-	/** The ID (logical timestamp) of the checkpoint */
+	/** The ID (logical timestamp) of the checkpoint. */
 	private final long checkpointID;
 
 	/** The timestamp when the checkpoint was triggered. */
@@ -91,21 +83,19 @@ public class CompletedCheckpoint implements Serializable {
 	/** The duration of the checkpoint (completion timestamp - trigger timestamp). */
 	private final long duration;
 
-	/** States of the different operator groups belonging to this checkpoint */
+	/** States of the different operator groups belonging to this checkpoint. */
 	private final Map<OperatorID, OperatorState> operatorStates;
 
 	/** Properties for this checkpoint. */
 	private final CheckpointProperties props;
 
-	/** States that were created by a hook on the master (in the checkpoint coordinator) */
+	/** States that were created by a hook on the master (in the checkpoint coordinator). */
 	private final Collection<MasterState> masterHookStates;
 
-	/** The state handle to the externalized meta data, if the metadata has been externalized */
-	@Nullable
-	private final StreamStateHandle externalizedMetadata;
+	/** The state handle to the externalized meta data. */
+	private final StreamStateHandle metadataHandle;
 
-	/** External pointer to the completed checkpoint (for example file path) if externalized; null otherwise. */
-	@Nullable
+	/** External pointer to the completed checkpoint (for example file path). */
 	private final String externalPointer;
 
 	/** Optional stats tracker callback for discard. */
@@ -122,19 +112,13 @@ public class CompletedCheckpoint implements Serializable {
 			Map<OperatorID, OperatorState> operatorStates,
 			@Nullable Collection<MasterState> masterHookStates,
 			CheckpointProperties props,
-			@Nullable StreamStateHandle externalizedMetadata,
-			@Nullable String externalPointer) {
+			StreamStateHandle metadataHandle,
+			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;
@@ -148,8 +132,8 @@ public class CompletedCheckpoint implements Serializable {
 				new ArrayList<>(masterHookStates);
 
 		this.props = checkNotNull(props);
-		this.externalizedMetadata = externalizedMetadata;
-		this.externalPointer = externalPointer;
+		this.metadataHandle = checkNotNull(metadataHandle);
+		this.externalPointer = checkNotNull(externalPointer);
 	}
 
 	// ------------------------------------------------------------------------
@@ -216,12 +200,10 @@ public class CompletedCheckpoint implements Serializable {
 			Exception exception = null;
 
 			// drop the metadata, if we have some
-			if (externalizedMetadata != null) {
-				try {
-					externalizedMetadata.discardState();
-				} catch (Exception e) {
-					exception = e;
-				}
+			try {
+				metadataHandle.discardState();
+			} catch (Exception e) {
+				exception = e;
 			}
 
 			// discard private state objects
@@ -263,16 +245,10 @@ public class CompletedCheckpoint implements Serializable {
 		return Collections.unmodifiableCollection(masterHookStates);
 	}
 
-	public boolean isExternalized() {
-		return externalizedMetadata != null;
+	public StreamStateHandle getMetadataHandle() {
+		return metadataHandle;
 	}
 
-	@Nullable
-	public StreamStateHandle getExternalizedMetadata() {
-		return externalizedMetadata;
-	}
-
-	@Nullable
 	public String getExternalPointer() {
 		return externalPointer;
 	}
@@ -314,10 +290,7 @@ public class CompletedCheckpoint implements Serializable {
 
 		CompletedCheckpoint that = (CompletedCheckpoint) o;
 
-		if (checkpointID != that.checkpointID) {
-			return false;
-		}
-		return job.equals(that.job);
+		return checkpointID == that.checkpointID && job.equals(that.job);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 a9b6d4d..5e8559d 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,14 +20,14 @@ package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.checkpoint.savepoint.Savepoint;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
 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;
 
@@ -35,7 +35,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -56,7 +55,7 @@ import static org.apache.flink.util.Preconditions.checkState;
  * A pending checkpoint is a checkpoint that has been started, but has not been
  * acknowledged by all tasks that need to acknowledge it. Once all tasks have
  * acknowledged it, it becomes a {@link CompletedCheckpoint}.
- * 
+ *
  * <p>Note that the pending checkpoint, as well as the successful checkpoint keep the
  * state handles always as serialized values, never as actual values.
  */
@@ -74,7 +73,7 @@ public class PendingCheckpoint {
 
 	// ------------------------------------------------------------------------
 
-	/** The PendingCheckpoint logs to the same logger as the CheckpointCoordinator */
+	/** The PendingCheckpoint logs to the same logger as the CheckpointCoordinator. */
 	private static final Logger LOG = LoggerFactory.getLogger(CheckpointCoordinator.class);
 
 	private final Object lock = new Object();
@@ -91,20 +90,19 @@ public class PendingCheckpoint {
 
 	private final List<MasterState> masterState;
 
-	/** Set of acknowledged tasks */
+	/** Set of acknowledged tasks. */
 	private final Set<ExecutionAttemptID> acknowledgedTasks;
 
-	/** The checkpoint properties. If the checkpoint should be persisted
-	 * externally, it happens in {@link #finalizeCheckpointExternalized()}. */
+	/** The checkpoint properties. */
 	private final CheckpointProperties props;
 
-	/** Target directory to potentially persist checkpoint to; <code>null</code> if none configured. */
-	private final String targetDirectory;
+	/** Target storage location to persist the checkpoint metadata to. */
+	private final CheckpointStorageLocation targetLocation;
 
 	/** The promise to fulfill once the checkpoint has been completed. */
 	private final CompletableFuture<CompletedCheckpoint> onCompletionPromise;
 
-	/** The executor for potentially blocking I/O operations, like state disposal */
+	/** The executor for potentially blocking I/O operations, like state disposal. */
 	private final Executor executor;
 
 	private int numAcknowledgedTasks;
@@ -125,14 +123,9 @@ public class PendingCheckpoint {
 			long checkpointTimestamp,
 			Map<ExecutionAttemptID, ExecutionVertex> verticesToConfirm,
 			CheckpointProperties props,
-			String targetDirectory,
+			CheckpointStorageLocation targetLocation,
 			Executor executor) {
 
-		// Sanity check
-		if (props.externalizeCheckpoint() && targetDirectory == null) {
-			throw new NullPointerException("No target directory specified to persist checkpoint to.");
-		}
-
 		checkArgument(verticesToConfirm.size() > 0,
 				"Checkpoint needs at least one vertex that commits the checkpoint");
 
@@ -141,7 +134,7 @@ public class PendingCheckpoint {
 		this.checkpointTimestamp = checkpointTimestamp;
 		this.notYetAcknowledgedTasks = checkNotNull(verticesToConfirm);
 		this.props = checkNotNull(props);
-		this.targetDirectory = targetDirectory;
+		this.targetLocation = checkNotNull(targetLocation);
 		this.executor = Preconditions.checkNotNull(executor);
 
 		this.operatorStates = new HashMap<>();
@@ -195,7 +188,7 @@ public class PendingCheckpoint {
 	/**
 	 * Checks whether this checkpoint can be subsumed or whether it should always continue, regardless
 	 * of newer checkpoints in progress.
-	 * 
+	 *
 	 * @return True if the checkpoint can be subsumed, false otherwise.
 	 */
 	public boolean canBeSubsumed() {
@@ -207,10 +200,6 @@ public class PendingCheckpoint {
 		return props;
 	}
 
-	String getTargetDirectory() {
-		return targetDirectory;
-	}
-
 	/**
 	 * Sets the callback for tracking this pending checkpoint.
 	 *
@@ -223,7 +212,7 @@ public class PendingCheckpoint {
 	/**
 	 * Sets the handle for the canceller to this pending checkpoint. This method fails
 	 * with an exception if a handle has already been set.
-	 * 
+	 *
 	 * @return true, if the handle was set, false, if the checkpoint is already disposed;
 	 */
 	public boolean setCancellerHandle(ScheduledFuture<?> cancellerHandle) {
@@ -255,103 +244,60 @@ public class PendingCheckpoint {
 		return onCompletionPromise;
 	}
 
-	public CompletedCheckpoint finalizeCheckpointExternalized() throws IOException {
+	public CompletedCheckpoint finalizeCheckpoint() throws IOException {
 
 		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 {
-				// externalize the metadata
+				// write out the metadata
 				final Savepoint savepoint = new SavepointV2(checkpointId, operatorStates.values(), masterState);
+				final StreamStateHandle metadataHandle;
 
-				// 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
-
-				// 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();
+				try (CheckpointStateOutputStream out = targetLocation.createMetadataOutputStream()) {
+					Checkpoints.storeCheckpointMetadata(savepoint, out);
+					metadataHandle = out.closeAndGetHandle();
+				}
 
-					return finalizeInternal(metadataHandle, externalPointer);
+				final String externalPointer = targetLocation.markCheckpointAsFinished();
+
+				CompletedCheckpoint completed = new CompletedCheckpoint(
+						jobId,
+						checkpointId,
+						checkpointTimestamp,
+						System.currentTimeMillis(),
+						operatorStates,
+						masterState,
+						props,
+						metadataHandle,
+						externalPointer);
+
+				onCompletionPromise.complete(completed);
+
+				// to prevent null-pointers from concurrent modification, copy reference onto stack
+				PendingCheckpointStats statsCallback = this.statsCallback;
+				if (statsCallback != null) {
+					// Finalize the statsCallback and give the completed checkpoint a
+					// callback for discards.
+					CompletedCheckpointStats.DiscardCallback discardCallback =
+							statsCallback.reportCompletedCheckpoint(externalPointer);
+					completed.setDiscardCallback(discardCallback);
 				}
-			}
-			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.");
+				// mark this pending checkpoint as disposed, but do NOT drop the state
+				dispose(false);
 
-			// make sure we fulfill the promise with an exception if something fails
-			try {
-				// finalize without external metadata
-				return finalizeInternal(null, null);
+				return completed;
 			}
 			catch (Throwable t) {
 				onCompletionPromise.completeExceptionally(t);
-				ExceptionUtils.rethrow(t);
+				ExceptionUtils.rethrowIOException(t);
 				return null; // silence the compiler
 			}
 		}
 	}
 
-	@GuardedBy("lock")
-	private CompletedCheckpoint finalizeInternal(
-			@Nullable StreamStateHandle externalMetadata,
-			@Nullable String externalPointer) {
-
-		assert(Thread.holdsLock(lock));
-
-		CompletedCheckpoint completed = new CompletedCheckpoint(
-				jobId,
-				checkpointId,
-				checkpointTimestamp,
-				System.currentTimeMillis(),
-				operatorStates,
-				masterState,
-				props,
-				externalMetadata,
-				externalPointer);
-
-		onCompletionPromise.complete(completed);
-
-		// to prevent null-pointers from concurrent modification, copy reference onto stack
-		PendingCheckpointStats statsCallback = this.statsCallback;
-		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;
-	}
-
 	/**
 	 * Acknowledges the task with the given execution attempt id and the given subtask state.
 	 *
@@ -528,6 +474,7 @@ public class PendingCheckpoint {
 							// unregistered shared states are still considered private at this point.
 							try {
 								StateUtil.bestEffortDiscardAllStateObjects(operatorStates.values());
+								targetLocation.disposeOnFailure();
 							} catch (Throwable t) {
 								LOG.warn("Could not properly dispose the private states in the pending checkpoint {} of job {}.",
 									checkpointId, jobId, t);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java
index a7cf4b5..d7966e6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java
@@ -34,8 +34,7 @@ import java.util.Collection;
  * we allow different savepoint implementations (see subclasses of this
  * interface).
  *
- * <p>Savepoints are serialized via a {@link SavepointSerializer} and stored
- * via a {@link SavepointStore}.
+ * <p>Savepoints are serialized via a {@link SavepointSerializer}.
  */
 public interface Savepoint extends Versioned {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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
deleted file mode 100644
index 31d9124..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.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.OperatorState;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.state.StreamStateHandle;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The SavepointLoader is a utility to load and verify a Savepoint, and to create a checkpoint from it.
- */
-public class SavepointLoader {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SavepointLoader.class);
-
-	/**
-	 * Loads a savepoint back as a {@link CompletedCheckpoint}.
-	 *
-	 * <p>This method verifies that tasks and parallelism still match the savepoint parameters.
-	 *
-	 * @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 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 IOException             If savepoint store failure
-	 */
-	public static CompletedCheckpoint loadAndValidateSavepoint(
-			JobID jobId,
-			Map<JobVertexID, ExecutionJobVertex> tasks,
-			String savepointPath,
-			ClassLoader classLoader,
-			boolean allowNonRestoredState) throws IOException {
-
-		// (1) load the savepoint
-		final Tuple2<Savepoint, StreamStateHandle> savepointAndHandle = 
-				SavepointStore.loadSavepointWithHandle(savepointPath, classLoader);
-
-		Savepoint savepoint = savepointAndHandle.f0;
-		final StreamStateHandle metadataHandle = savepointAndHandle.f1;
-
-		if (savepoint.getTaskStates() != null) {
-			savepoint = SavepointV2.convertToOperatorStateSavepointV2(tasks, savepoint);
-		}
-		// generate mapping from operator to task
-		Map<OperatorID, ExecutionJobVertex> operatorToJobVertexMapping = new HashMap<>();
-		for (ExecutionJobVertex task : tasks.values()) {
-			for (OperatorID operatorID : task.getOperatorIDs()) {
-				operatorToJobVertexMapping.put(operatorID, task);
-			}
-		}
-
-		// (2) validate it (parallelism, etc)
-		boolean expandedToLegacyIds = false;
-
-		HashMap<OperatorID, OperatorState> operatorStates = new HashMap<>(savepoint.getOperatorStates().size());
-		for (OperatorState operatorState : savepoint.getOperatorStates()) {
-
-			ExecutionJobVertex executionJobVertex = operatorToJobVertexMapping.get(operatorState.getOperatorID());
-
-			// on the first time we can not find the execution job vertex for an id, we also consider alternative ids,
-			// for example as generated from older flink versions, to provide backwards compatibility.
-			if (executionJobVertex == null && !expandedToLegacyIds) {
-				operatorToJobVertexMapping = ExecutionJobVertex.includeAlternativeOperatorIDs(operatorToJobVertexMapping);
-				executionJobVertex = operatorToJobVertexMapping.get(operatorState.getOperatorID());
-				expandedToLegacyIds = true;
-				LOG.info("Could not find ExecutionJobVertex. Including user-defined OperatorIDs in search.");
-			}
-
-			if (executionJobVertex != null) {
-
-				if (executionJobVertex.getMaxParallelism() == operatorState.getMaxParallelism()
-						|| !executionJobVertex.isMaxParallelismConfigured()) {
-					operatorStates.put(operatorState.getOperatorID(), operatorState);
-				} else {
-					String msg = String.format("Failed to rollback to savepoint %s. " +
-									"Max parallelism mismatch between savepoint state and new program. " +
-									"Cannot map operator %s with max parallelism %d to new program with " +
-									"max parallelism %d. This indicates that the program has been changed " +
-									"in a non-compatible way after the savepoint.",
-							savepoint,
-							operatorState.getOperatorID(),
-							operatorState.getMaxParallelism(),
-							executionJobVertex.getMaxParallelism());
-
-					throw new IllegalStateException(msg);
-				}
-			} else if (allowNonRestoredState) {
-				LOG.info("Skipping savepoint state for operator {}.", operatorState.getOperatorID());
-			} else {
-				for (OperatorSubtaskState operatorSubtaskState : operatorState.getStates()) {
-					if (operatorSubtaskState.hasState()) {
-						String msg = String.format("Failed to rollback to savepoint %s. " +
-								"Cannot map savepoint state for operator %s to the new program, " +
-								"because the operator is not available in the new program. If " +
-								"you want to allow to skip this, you can set the --allowNonRestoredState " +
-								"option on the CLI.",
-							savepointPath, operatorState.getOperatorID());
-
-						throw new IllegalStateException(msg);
-					}
-				}
-				LOG.info("Skipping empty savepoint state for operator {}.", operatorState.getOperatorID());
-			}
-		}
-
-		// (3) convert to checkpoint so the system can fall back to it
-		CheckpointProperties props = CheckpointProperties.forStandardSavepoint();
-
-		return new CompletedCheckpoint(
-			jobId,
-			savepoint.getCheckpointId(),
-			0L,
-			0L,
-			operatorStates,
-			savepoint.getMasterStates(),
-			props,
-			metadataHandle,
-			savepointPath);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/** This class is not meant to be instantiated */
-	private SavepointLoader() {}
-}


[11/17] flink git commit: [FLINK-7925] [checkpoints] Add CheckpointingOptions

Posted by se...@apache.org.
[FLINK-7925] [checkpoints] Add CheckpointingOptions

The CheckpointingOptions consolidate all checkpointing and state backend-related
settings that were previously split across different classes.


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

Branch: refs/heads/master
Commit: e52db8bc411e93c245cc78a278854f2653e5f384
Parents: 03c797a
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 25 17:30:14 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../flink/client/cli/CliFrontendParser.java     |  4 +-
 .../connectors/fs/RollingSinkSecuredITCase.java |  4 +-
 .../configuration/CheckpointingOptions.java     | 92 ++++++++++++++++++++
 .../apache/flink/configuration/CoreOptions.java | 24 -----
 .../runtime/webmonitor/WebRuntimeMonitor.java   |  4 +-
 .../checkpoint/CheckpointCoordinator.java       |  4 +-
 .../executiongraph/ExecutionGraphBuilder.java   | 50 ++++++-----
 .../job/savepoints/SavepointHandlers.java       |  6 +-
 .../JobCancellationWithSavepointHandlers.java   |  4 +-
 .../runtime/webmonitor/WebMonitorEndpoint.java  |  4 +-
 .../flink/runtime/jobmanager/JobManager.scala   |  9 +-
 .../ExecutionGraphDeploymentTest.java           | 10 +--
 .../runtime/jobmanager/JobManagerTest.java      |  6 +-
 ...obCancellationWithSavepointHandlersTest.java |  4 +-
 .../runtime/testutils/ZooKeeperTestUtils.java   |  7 +-
 .../api/environment/CheckpointConfig.java       |  3 +-
 .../streaming/runtime/tasks/StreamTaskTest.java |  6 +-
 .../ExternalizedCheckpointITCase.java           |  9 +-
 .../test/checkpointing/RescalingITCase.java     |  9 +-
 .../test/checkpointing/SavepointITCase.java     | 32 +++----
 .../utils/SavepointMigrationTestBase.java       | 13 +--
 .../test/classloading/ClassLoaderITCase.java    |  9 +-
 .../JobManagerHACheckpointRecoveryITCase.java   |  6 +-
 .../state/operator/restore/keyed/KeyedJob.java  |  4 +-
 .../operator/restore/unkeyed/NonKeyedJob.java   |  4 +-
 .../StatefulJobSavepointMigrationITCase.scala   | 24 ++---
 .../flink/yarn/YARNHighAvailabilityITCase.java  |  7 +-
 27 files changed, 209 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 402b4ae..e5d550f 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.client.cli;
 
-import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.DefaultParser;
@@ -93,7 +93,7 @@ public class CliFrontendParser {
 	static final Option CANCEL_WITH_SAVEPOINT_OPTION = new Option(
 			"s", "withSavepoint", true, "Trigger savepoint and cancel job. The target " +
 			"directory is optional. If no directory is specified, the configured default " +
-			"directory (" + CoreOptions.SAVEPOINT_DIRECTORY.key() + ") is used.");
+			"directory (" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + ") is used.");
 
 	static {
 		HELP_OPTION.setRequired(false);

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
index b76d087..b9564ee 100644
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.connectors.fs;
 
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.runtime.security.SecurityConfiguration;
@@ -216,7 +216,7 @@ public class RollingSinkSecuredITCase extends RollingSinkITCase {
 			result.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false);
 			result.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
 			result.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-			result.setString(CoreOptions.STATE_BACKEND, "filesystem");
+			result.setString(CheckpointingOptions.STATE_BACKEND, "filesystem");
 			result.setString(HighAvailabilityOptions.HA_ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints");
 			result.setString(HighAvailabilityOptions.HA_STORAGE_PATH, hdfsURI + "/flink/recovery");
 			result.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints");

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
new file mode 100644
index 0000000..1825ba3
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -0,0 +1,92 @@
+/*
+ * 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.configuration;
+
+/**
+ * A collection of all configuration options that relate to checkpoints
+ * and savepoints.
+ */
+public class CheckpointingOptions {
+
+	// ------------------------------------------------------------------------
+	//  general checkpoint and state backend options
+	// ------------------------------------------------------------------------
+
+	/** The state backend to be used to store and checkpoint state. */
+	public static final ConfigOption<String> STATE_BACKEND = ConfigOptions
+			.key("state.backend")
+			.noDefaultValue();
+
+	/** The maximum number of completed checkpoints to retain.*/
+	public static final ConfigOption<Integer> MAX_RETAINED_CHECKPOINTS = ConfigOptions
+			.key("state.checkpoints.num-retained")
+			.defaultValue(1);
+
+	/** Option whether the state backend should use an asynchronous snapshot method where
+	 * possible and configurable.
+	 *
+	 * <p>Some state backends may not support asynchronous snapshots, or only support
+	 * asynchronous snapshots, and ignore this option. */
+	public static final ConfigOption<Boolean> ASYNC_SNAPSHOTS = ConfigOptions
+			.key("state.backend.async")
+			.defaultValue(true);
+
+	/** Option whether the state backend should create incremental checkpoints,
+	 * if possible. For an incremental checkpoint, only a diff from the previous
+	 * checkpoint is stored, rather than the complete checkpoint state.
+	 *
+	 * <p>Some state backends may not support incremental checkpoints and ignore
+	 * this option.*/
+	public static final ConfigOption<Boolean> INCREMENTAL_CHECKPOINTS = ConfigOptions
+			.key("state.backend.incremental")
+			.defaultValue(false);
+
+	// ------------------------------------------------------------------------
+	//  Options specific to the file-system-based state backends
+	// ------------------------------------------------------------------------
+
+	/** The default directory for savepoints. Used by the state backends that write
+	 * savepoints to file systems (MemoryStateBackend, FsStateBackend, RocksDBStateBackend). */
+	public static final ConfigOption<String> SAVEPOINT_DIRECTORY = ConfigOptions
+			.key("state.savepoints.dir")
+			.noDefaultValue()
+			.withDeprecatedKeys("savepoints.state.backend.fs.dir");
+
+	/** The default directory used for checkpoints. Used by the state backends that write
+	 * checkpoints to file systems (MemoryStateBackend, FsStateBackend, RocksDBStateBackend). */
+	public static final ConfigOption<String> CHECKPOINTS_DIRECTORY = ConfigOptions
+			.key("state.checkpoints.dir")
+			.noDefaultValue();
+
+	/** The minimum size of state data files. All state chunks smaller than that
+	 * are stored inline in the root checkpoint metadata file. */
+	public static final ConfigOption<Integer> FS_SMALL_FILE_THRESHOLD = ConfigOptions
+			.key("state.backend.fs.memory-threshold")
+			.defaultValue(1024);
+
+	// ------------------------------------------------------------------------
+	//  Options specific to the RocksDB state backend
+	// ------------------------------------------------------------------------
+
+	/** The local directory (on the TaskManager) where RocksDB puts its files. */
+	public static final ConfigOption<String> ROCKSDB_LOCAL_DIRECTORIES = ConfigOptions
+			.key("state.backend.rocksdb.localdir")
+			.noDefaultValue()
+			.withDeprecatedKeys("state.backend.rocksdb.checkpointdir");
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
index 4592608..f31ad8c 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
@@ -120,30 +120,6 @@ public class CoreOptions {
 		.defaultValue(-1);
 
 	// ------------------------------------------------------------------------
-	//  checkpoints / fault tolerance
-	// ------------------------------------------------------------------------
-
-	public static final ConfigOption<String> STATE_BACKEND = ConfigOptions
-		.key("state.backend")
-		.noDefaultValue();
-
-	/** The maximum number of completed checkpoint instances to retain.*/
-	public static final ConfigOption<Integer> MAX_RETAINED_CHECKPOINTS = ConfigOptions
-		.key("state.checkpoints.num-retained")
-		.defaultValue(1);
-
-	/** The default directory for savepoints. */
-	public static final ConfigOption<String> SAVEPOINT_DIRECTORY = ConfigOptions
-		.key("state.savepoints.dir")
-		.noDefaultValue()
-		.withDeprecatedKeys("savepoints.state.backend.fs.dir");
-
-	/** The default directory used for persistent checkpoints. */
-	public static final ConfigOption<String> CHECKPOINTS_DIRECTORY = ConfigOptions
-		.key("state.checkpoints.dir")
-		.noDefaultValue();
-
-	// ------------------------------------------------------------------------
 	//  file systems
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 445c61c..7d232a5 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -19,8 +19,8 @@
 package org.apache.flink.runtime.webmonitor;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.runtime.concurrent.ScheduledExecutor;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
@@ -249,7 +249,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		}
 		metricFetcher = new MetricFetcher(retriever, queryServiceRetriever, scheduledExecutor, timeout);
 
-		String defaultSavepointDir = config.getString(CoreOptions.SAVEPOINT_DIRECTORY);
+		String defaultSavepointDir = config.getString(CheckpointingOptions.SAVEPOINT_DIRECTORY);
 
 		JobCancellationWithSavepointHandlers cancelWithSavepoint = new JobCancellationWithSavepointHandlers(executionGraphCache, scheduledExecutor, defaultSavepointDir);
 		RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler());

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 824563f..a04e34e 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
@@ -22,7 +22,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.runtime.checkpoint.hooks.MasterHooks;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointLoader;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
@@ -208,7 +208,7 @@ public class CheckpointCoordinator {
 		if (externalizeSettings.externalizeCheckpoints() && checkpointDirectory == null) {
 			throw new IllegalStateException("CheckpointConfig says to persist periodic " +
 					"checkpoints, but no checkpoint directory has been configured. You can " +
-					"configure configure one via key '" + CoreOptions.CHECKPOINTS_DIRECTORY.key() + "'.");
+					"configure configure one via key '" + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key() + "'.");
 		}
 
 		// max "in between duration" can be one year - this is to prevent numeric overflows

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 34ba3df..47948a9 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
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.executiongraph;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.metrics.MetricGroup;
@@ -50,8 +50,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
 import org.apache.flink.util.DynamicCodeLoadingException;
 import org.apache.flink.util.SerializedValue;
 
@@ -199,17 +199,17 @@ public class ExecutionGraphBuilder {
 			CheckpointIDCounter checkpointIdCounter;
 			try {
 				int maxNumberOfCheckpointsToRetain = jobManagerConfig.getInteger(
-					CoreOptions.MAX_RETAINED_CHECKPOINTS);
+						CheckpointingOptions.MAX_RETAINED_CHECKPOINTS);
 
 				if (maxNumberOfCheckpointsToRetain <= 0) {
 					// warning and use 1 as the default value if the setting in
 					// state.checkpoints.max-retained-checkpoints is not greater than 0.
 					log.warn("The setting for '{} : {}' is invalid. Using default value of {}",
-							CoreOptions.MAX_RETAINED_CHECKPOINTS.key(),
+							CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(),
 							maxNumberOfCheckpointsToRetain,
-							CoreOptions.MAX_RETAINED_CHECKPOINTS.defaultValue());
+							CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue());
 
-					maxNumberOfCheckpointsToRetain = CoreOptions.MAX_RETAINED_CHECKPOINTS.defaultValue();
+					maxNumberOfCheckpointsToRetain = CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue();
 				}
 
 				completedCheckpoints = recoveryFactory.createCheckpointStore(jobId, maxNumberOfCheckpointsToRetain, classLoader);
@@ -229,29 +229,31 @@ public class ExecutionGraphBuilder {
 					metrics);
 
 			// The default directory for externalized checkpoints
-			String externalizedCheckpointsDir = jobManagerConfig.getString(CoreOptions.CHECKPOINTS_DIRECTORY);
+			String externalizedCheckpointsDir = jobManagerConfig.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY);
 
-			// load the state backend for checkpoint metadata.
-			// if specified in the application, use from there, otherwise load from configuration
-			final StateBackend metadataBackend;
+			// load the state backend from the application settings
+			final StateBackend applicationConfiguredBackend;
+			final SerializedValue<StateBackend> serializedAppConfigured = snapshotSettings.getDefaultStateBackend();
 
-			final SerializedValue<StateBackend> applicationConfiguredBackend = snapshotSettings.getDefaultStateBackend();
-			if (applicationConfiguredBackend != null) {
+			if (serializedAppConfigured == null) {
+				applicationConfiguredBackend = null;
+			}
+			else {
 				try {
-					metadataBackend = applicationConfiguredBackend.deserializeValue(classLoader);
+					applicationConfiguredBackend = serializedAppConfigured.deserializeValue(classLoader);
 				} catch (IOException | ClassNotFoundException e) {
-					throw new JobExecutionException(jobId, "Could not instantiate configured state backend.", e);
+					throw new JobExecutionException(jobId, 
+							"Could not deserialize application-defined state backend.", e);
 				}
+			}
 
-				log.info("Using application-defined state backend for checkpoint/savepoint metadata: {}.",
-					metadataBackend);
-			} else {
-				try {
-					metadataBackend = AbstractStateBackend
-							.loadStateBackendFromConfigOrCreateDefault(jobManagerConfig, classLoader, log);
-				} catch (IllegalConfigurationException | IOException | DynamicCodeLoadingException e) {
-					throw new JobExecutionException(jobId, "Could not instantiate configured state backend", e);
-				}
+			final StateBackend rootBackend;
+			try {
+				rootBackend = StateBackendLoader.fromApplicationOrConfigOrDefault(
+						applicationConfiguredBackend, jobManagerConfig, classLoader, log);
+			}
+			catch (IllegalConfigurationException | IOException | DynamicCodeLoadingException e) {
+				throw new JobExecutionException(jobId, "Could not instantiate configured state backend", e);
 			}
 
 			// instantiate the user-defined checkpoint hooks
@@ -301,7 +303,7 @@ public class ExecutionGraphBuilder {
 				checkpointIdCounter,
 				completedCheckpoints,
 				externalizedCheckpointsDir,
-				metadataBackend,
+				rootBackend,
 				checkpointStatsTracker);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
index 6b437d0..ade2b54 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.rest.handler.job.savepoints;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.rest.NotFoundException;
 import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
@@ -79,7 +79,7 @@ import static java.util.Objects.requireNonNull;
  * { "target-directory": "/tmp" }
  * </pre>
  * If the body is omitted, or the field {@code target-property} is {@code null}, the default
- * savepoint directory as specified by {@link CoreOptions#SAVEPOINT_DIRECTORY} will be used.
+ * savepoint directory as specified by {@link CheckpointingOptions#SAVEPOINT_DIRECTORY} will be used.
  * As written above, the response will contain a request id, e.g.,
  * <pre>
  * { "request-id": "7d273f5a62eb4730b9dea8e833733c1e" }
@@ -146,7 +146,7 @@ public class SavepointHandlers {
 				return FutureUtils.completedExceptionally(
 					new RestHandlerException(
 						String.format("Config key [%s] is not set. Property [%s] must be provided.",
-							CoreOptions.SAVEPOINT_DIRECTORY.key(),
+							CheckpointingOptions.SAVEPOINT_DIRECTORY.key(),
 							SavepointTriggerRequestBody.FIELD_NAME_TARGET_DIRECTORY),
 						HttpResponseStatus.BAD_REQUEST));
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
index 12f27dd..93564e9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.rest.handler.legacy;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
@@ -166,7 +166,7 @@ public class JobCancellationWithSavepointHandlers {
 									throw new IllegalStateException("No savepoint directory configured. " +
 										"You can either specify a directory when triggering this savepoint or " +
 										"configure a cluster-wide default via key '" +
-										CoreOptions.SAVEPOINT_DIRECTORY.key() + "'.");
+											CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.");
 								} else {
 									targetDirectory = defaultSavepointDirectory;
 								}

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
index 7ef17d8..f0cfb5e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
@@ -20,8 +20,8 @@ package org.apache.flink.runtime.webmonitor;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rest.RestServerEndpoint;
 import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
@@ -356,7 +356,7 @@ public class WebMonitorEndpoint<T extends RestfulGateway> extends RestServerEndp
 			timeout,
 			responseHeaders);
 
-		final String defaultSavepointDir = clusterConfiguration.getString(CoreOptions.SAVEPOINT_DIRECTORY);
+		final String defaultSavepointDir = clusterConfiguration.getString(CheckpointingOptions.SAVEPOINT_DIRECTORY);
 
 		final SavepointHandlers savepointHandlers = new SavepointHandlers(defaultSavepointDir);
 		final SavepointHandlers.SavepointTriggerHandler savepointTriggerHandler = savepointHandlers.new SavepointTriggerHandler(

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 5f82159..0f8033d 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
@@ -156,7 +156,8 @@ class JobManager(
   var futuresToComplete: Option[Seq[Future[Unit]]] = None
 
   /** The default directory for savepoints. */
-  val defaultSavepointDir: String = flinkConfiguration.getString(CoreOptions.SAVEPOINT_DIRECTORY)
+  val defaultSavepointDir: String = 
+    flinkConfiguration.getString(CheckpointingOptions.SAVEPOINT_DIRECTORY)
 
   /** The resource manager actor responsible for allocating and managing task manager resources. */
   var currentResourceManager: Option[ActorRef] = None
@@ -564,7 +565,7 @@ class JobManager(
           sender ! decorateMessage(CancellationFailure(jobId, new IllegalStateException(
             "No savepoint directory configured. You can either specify a directory " +
               "while cancelling via -s :targetDirectory or configure a cluster-wide " +
-              "default via key '" + CoreOptions.SAVEPOINT_DIRECTORY.key() + "'.")))
+              "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")))
         } else {
           log.info(s"Trying to cancel job $jobId with savepoint to $targetDirectory")
 
@@ -750,13 +751,13 @@ class JobManager(
             val senderRef = sender()
             try {
               val targetDirectory : String = savepointDirectory.getOrElse(
-                flinkConfiguration.getString(CoreOptions.SAVEPOINT_DIRECTORY))
+                flinkConfiguration.getString(CheckpointingOptions.SAVEPOINT_DIRECTORY))
 
               if (targetDirectory == null) {
                 throw new IllegalStateException("No savepoint directory configured. " +
                   "You can either specify a directory when triggering this savepoint or " +
                   "configure a cluster-wide default via key '" +
-                  CoreOptions.SAVEPOINT_DIRECTORY.key() + "'.")
+                  CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")
               }
 
               // Do this async, because checkpoint coordinator operations can

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index e869625..12e9b5d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.IntCounter;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -479,7 +479,7 @@ public class ExecutionGraphDeploymentTest extends TestLogger {
 
 		final ExecutionGraph eg = createExecutionGraph(jobManagerConfig);
 
-		assertEquals(CoreOptions.MAX_RETAINED_CHECKPOINTS.defaultValue().intValue(),
+		assertEquals(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue().intValue(),
 				eg.getCheckpointCoordinator().getCheckpointStore().getMaxNumberOfRetainedCheckpoints());
 	}
 
@@ -488,7 +488,7 @@ public class ExecutionGraphDeploymentTest extends TestLogger {
 
 		final int maxNumberOfCheckpointsToRetain = 10;
 		final Configuration jobManagerConfig = new Configuration();
-		jobManagerConfig.setInteger(CoreOptions.MAX_RETAINED_CHECKPOINTS,
+		jobManagerConfig.setInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS,
 			maxNumberOfCheckpointsToRetain);
 
 		final ExecutionGraph eg = createExecutionGraph(jobManagerConfig);
@@ -554,7 +554,7 @@ public class ExecutionGraphDeploymentTest extends TestLogger {
 		final int negativeMaxNumberOfCheckpointsToRetain = -10;
 
 		final Configuration jobManagerConfig = new Configuration();
-		jobManagerConfig.setInteger(CoreOptions.MAX_RETAINED_CHECKPOINTS,
+		jobManagerConfig.setInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS,
 			negativeMaxNumberOfCheckpointsToRetain);
 
 		final ExecutionGraph eg = createExecutionGraph(jobManagerConfig);
@@ -562,7 +562,7 @@ public class ExecutionGraphDeploymentTest extends TestLogger {
 		assertNotEquals(negativeMaxNumberOfCheckpointsToRetain,
 			eg.getCheckpointCoordinator().getCheckpointStore().getMaxNumberOfRetainedCheckpoints());
 
-		assertEquals(CoreOptions.MAX_RETAINED_CHECKPOINTS.defaultValue().intValue(),
+		assertEquals(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue().intValue(),
 			eg.getCheckpointCoordinator().getCheckpointStore().getMaxNumberOfRetainedCheckpoints());
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 ecf2ae3..01d2346 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
@@ -20,9 +20,9 @@ package org.apache.flink.runtime.jobmanager;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.AkkaOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.queryablestate.KvStateID;
@@ -830,7 +830,7 @@ public class JobManagerTest extends TestLogger {
 
 		FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
 		Configuration config = new Configuration();
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
 
 		ActorSystem actorSystem = null;
 		ActorGateway jobManager = null;
@@ -1157,7 +1157,7 @@ public class JobManagerTest extends TestLogger {
 
 		FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
 		Configuration config = new Configuration();
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
 
 		ActorSystem actorSystem = null;
 		ActorGateway jobManager = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
index 9c63673..9e801a3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.rest.handler.legacy;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
@@ -165,7 +165,7 @@ public class JobCancellationWithSavepointHandlersTest extends TestLogger {
 			fail("Did not throw expected test Exception");
 		} catch (Exception e) {
 			IllegalStateException cause = (IllegalStateException) e.getCause();
-			assertEquals(true, cause.getMessage().contains(CoreOptions.SAVEPOINT_DIRECTORY.key()));
+			assertEquals(true, cause.getMessage().contains(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
index 9af8aaf..e688e2f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
@@ -19,12 +19,11 @@
 package org.apache.flink.runtime.testutils;
 
 import org.apache.flink.configuration.AkkaOptions;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -82,8 +81,8 @@ public class ZooKeeperTestUtils {
 		config.setInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
 
 		// File system state backend
-		config.setString(CoreOptions.STATE_BACKEND, "FILESYSTEM");
-		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, fsStateHandlePath + "/checkpoints");
+		config.setString(CheckpointingOptions.STATE_BACKEND, "FILESYSTEM");
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, fsStateHandlePath + "/checkpoints");
 		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, fsStateHandlePath + "/recovery");
 
 		// Akka failure detection and execution retries

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
index 342d4a7..87c800d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.api.environment;
 
 import org.apache.flink.annotation.Public;
 import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.streaming.api.CheckpointingMode;
 
@@ -267,7 +266,7 @@ public class CheckpointConfig implements java.io.Serializable {
 	 * (terminating with job status {@link JobStatus#CANCELED}).
 	 *
 	 * <p>The target directory for externalized checkpoints is configured
-	 * via {@link CoreOptions#CHECKPOINTS_DIRECTORY}.
+	 * via {@link org.apache.flink.configuration.CheckpointingOptions#CHECKPOINTS_DIRECTORY}.
 	 *
 	 * @param cleanupMode Externalized checkpoint cleanup behaviour.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 46862f2..f492d9e 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
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.blob.BlobCacheService;
@@ -211,7 +211,7 @@ public class StreamTaskTest extends TestLogger {
 	@Test
 	public void testStateBackendLoadingAndClosing() throws Exception {
 		Configuration taskManagerConfig = new Configuration();
-		taskManagerConfig.setString(CoreOptions.STATE_BACKEND, MockStateBackend.class.getName());
+		taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, MockStateBackend.class.getName());
 
 		StreamConfig cfg = new StreamConfig(new Configuration());
 		cfg.setOperatorID(new OperatorID(4711L, 42L));
@@ -236,7 +236,7 @@ public class StreamTaskTest extends TestLogger {
 	@Test
 	public void testStateBackendClosingOnFailure() throws Exception {
 		Configuration taskManagerConfig = new Configuration();
-		taskManagerConfig.setString(CoreOptions.STATE_BACKEND, MockStateBackend.class.getName());
+		taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, MockStateBackend.class.getName());
 
 		StreamConfig cfg = new StreamConfig(new Configuration());
 		cfg.setOperatorID(new OperatorID(4711L, 42L));

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ExternalizedCheckpointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ExternalizedCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ExternalizedCheckpointITCase.java
index 6ba5ca4..f733e6d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ExternalizedCheckpointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ExternalizedCheckpointITCase.java
@@ -22,9 +22,9 @@ import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
@@ -32,7 +32,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.environment.CheckpointConfig;
@@ -150,9 +149,9 @@ public class ExternalizedCheckpointITCase extends TestLogger {
 
 		final File savepointDir = temporaryFolder.newFolder();
 
-		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, checkpointDir.toURI().toString());
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
-		config.setString(CoreOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
 
 		// ZooKeeper recovery mode?
 		if (zooKeeperQuorum != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
index 40b45f9..f874c8c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
@@ -28,9 +28,9 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.instance.ActorGateway;
@@ -40,7 +40,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
@@ -127,9 +126,9 @@ public class RescalingITCase extends TestLogger {
 			final File checkpointDir = temporaryFolder.newFolder();
 			final File savepointDir = temporaryFolder.newFolder();
 
-			config.setString(CoreOptions.STATE_BACKEND, currentBackend);
-			config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, checkpointDir.toURI().toString());
-			config.setString(CoreOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
+			config.setString(CheckpointingOptions.STATE_BACKEND, currentBackend);
+			config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+			config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
 
 			cluster = new TestingCluster(config);
 			cluster.start();

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 75f0aa4..2711870 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
@@ -28,9 +28,9 @@ import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -55,8 +55,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-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;
@@ -122,7 +120,7 @@ public class SavepointITCase extends TestLogger {
 	private static final Logger LOG = LoggerFactory.getLogger(SavepointITCase.class);
 
 	@Rule
-	public TemporaryFolder folder = new TemporaryFolder();
+	public final TemporaryFolder folder = new TemporaryFolder();
 
 	/**
 	 * Triggers a savepoint for a job that uses the FsStateBackend. We expect
@@ -166,10 +164,10 @@ public class SavepointITCase extends TestLogger {
 			}
 
 			// Use file based checkpoints
-			config.setString(CoreOptions.STATE_BACKEND, "filesystem");
-			config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, checkpointDir.toURI().toString());
-			config.setString(FsStateBackendFactory.MEMORY_THRESHOLD_CONF_KEY, "0");
-			config.setString(CoreOptions.SAVEPOINT_DIRECTORY, savepointRootDir.toURI().toString());
+			config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem");
+			config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+			config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0);
+			config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointRootDir.toURI().toString());
 
 			// Start Flink
 			flink = new TestingCluster(config);
@@ -434,8 +432,7 @@ public class SavepointITCase extends TestLogger {
 			final Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTaskManager);
-			config.setString(CoreOptions.SAVEPOINT_DIRECTORY,
-				savepointDir.toURI().toString());
+			config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
 
 			LOG.info("Flink configuration: " + config + ".");
 
@@ -501,8 +498,7 @@ public class SavepointITCase extends TestLogger {
 		final Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTaskManager);
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY,
-				savepointDir.toURI().toString());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
 
 		String savepointPath;
 
@@ -686,7 +682,7 @@ public class SavepointITCase extends TestLogger {
 			if (data == null) {
 				// We need this to be large, because we want to test with files
 				Random rand = new Random(getRuntimeContext().getIndexOfThisSubtask());
-				data = new byte[FsStateBackend.DEFAULT_FILE_STATE_THRESHOLD + 1];
+				data = new byte[CheckpointingOptions.FS_SMALL_FILE_THRESHOLD.defaultValue() + 1];
 				rand.nextBytes(data);
 			}
 		}
@@ -808,12 +804,10 @@ public class SavepointITCase extends TestLogger {
 			fail("Test setup failed: failed to create temporary directories.");
 		}
 
-		config.setString(CoreOptions.STATE_BACKEND, "filesystem");
-		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY,
-				checkpointDir.toURI().toString());
-		config.setString(FsStateBackendFactory.MEMORY_THRESHOLD_CONF_KEY, "0");
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY,
-				savepointDir.toURI().toString());
+		config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem");
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+		config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0);
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
 
 		TestingCluster cluster = new TestingCluster(config, false);
 		String savepointPath = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
index eccc7e9..b694e0c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
@@ -22,9 +22,9 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers;
 import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.runtime.instance.ActorGateway;
@@ -33,16 +33,17 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.util.TestBaseUtils;
 
 import org.apache.commons.io.FileUtils;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -107,10 +108,10 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		LOG.info("Created temporary checkpoint directory: " + checkpointDir + ".");
 		LOG.info("Created savepoint directory: " + savepointDir + ".");
 
-		config.setString(CoreOptions.STATE_BACKEND, "memory");
-		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, checkpointDir.toURI().toString());
-		config.setString(FsStateBackendFactory.MEMORY_THRESHOLD_CONF_KEY, "0");
-		config.setString("state.savepoints.dir", savepointDir.toURI().toString());
+		config.setString(CheckpointingOptions.STATE_BACKEND, "memory");
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
+		config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0);
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
 
 		cluster = TestBaseUtils.startCluster(config, false);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index 58caf5c..b357904 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -21,9 +21,9 @@ package org.apache.flink.test.classloading;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.client.JobCancellationException;
 import org.apache.flink.runtime.client.JobStatusMessage;
@@ -34,7 +34,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepointFail
 import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
@@ -114,12 +113,12 @@ public class ClassLoaderITCase extends TestLogger {
 		parallelism = 4;
 
 		// we need to use the "filesystem" state backend to ensure FLINK-2543 is not happening again.
-		config.setString(CoreOptions.STATE_BACKEND, "filesystem");
-		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY,
+		config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem");
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY,
 				FOLDER.newFolder().getAbsoluteFile().toURI().toString());
 
 		// Savepoint path
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY,
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY,
 				FOLDER.newFolder().getAbsoluteFile().toURI().toString());
 
 		testCluster = new TestingCluster(config, false);

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/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 cefadb4..16ea6d5 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
@@ -24,9 +24,9 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -362,11 +362,11 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		try {
 			Configuration config = new Configuration();
 
-			config.setInteger(CoreOptions.MAX_RETAINED_CHECKPOINTS, retainedCheckpoints);
+			config.setInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, retainedCheckpoints);
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
-			config.setString(CoreOptions.CHECKPOINTS_DIRECTORY, temporaryFolder.newFolder().toString());
+			config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, temporaryFolder.newFolder().toString());
 
 			String tmpFolderString = temporaryFolder.newFolder().toString();
 			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, tmpFolderString);

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
index 076feda..c53f80f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -59,7 +59,7 @@ public class KeyedJob {
 		String savepointsPath = pt.getRequired("savepoint-path");
 
 		Configuration config = new Configuration();
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY, savepointsPath);
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointsPath);
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(config);
 		env.enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE);

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
index 1b0ed45..0b5a21f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
@@ -22,8 +22,8 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -52,7 +52,7 @@ public class NonKeyedJob {
 		String savepointsPath = pt.getRequired("savepoint-path");
 
 		Configuration config = new Configuration();
-		config.setString(CoreOptions.SAVEPOINT_DIRECTORY, savepointsPath);
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointsPath);
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(config);
 		env.enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE);

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala
index 1e67042..435262c 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala
@@ -35,14 +35,14 @@ import org.apache.flink.streaming.api.watermark.Watermark
 import org.apache.flink.test.checkpointing.utils.SavepointMigrationTestBase
 import org.apache.flink.util.Collector
 import org.apache.flink.api.java.tuple.Tuple2
-import org.apache.flink.runtime.state.{AbstractStateBackend, FunctionInitializationContext, FunctionSnapshotContext}
+import org.apache.flink.runtime.state.{StateBackendLoader, FunctionInitializationContext, FunctionSnapshotContext}
 import org.apache.flink.api.scala._
 import org.apache.flink.api.scala.migration.CustomEnum.CustomEnum
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend
 import org.apache.flink.streaming.util.migration.MigrationVersion
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
-import org.junit.{Assume, Ignore, Test}
+import org.junit.{Ignore, Test}
 
 import scala.util.{Failure, Properties, Try}
 
@@ -51,17 +51,17 @@ object StatefulJobSavepointMigrationITCase {
   @Parameterized.Parameters(name = "Migrate Savepoint / Backend: {0}")
   def parameters: util.Collection[(MigrationVersion, String)] = {
     util.Arrays.asList(
-      (MigrationVersion.v1_2, AbstractStateBackend.MEMORY_STATE_BACKEND_NAME),
-      (MigrationVersion.v1_2, AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME),
-      (MigrationVersion.v1_3, AbstractStateBackend.MEMORY_STATE_BACKEND_NAME),
-      (MigrationVersion.v1_3, AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME))
+      (MigrationVersion.v1_2, StateBackendLoader.MEMORY_STATE_BACKEND_NAME),
+      (MigrationVersion.v1_2, StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME),
+      (MigrationVersion.v1_3, StateBackendLoader.MEMORY_STATE_BACKEND_NAME),
+      (MigrationVersion.v1_3, StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME))
   }
 
   // TODO to generate savepoints for a specific Flink version / backend type,
   // TODO change these values accordingly, e.g. to generate for 1.3 with RocksDB,
-  // TODO set as (MigrationVersion.v1_3, AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME)
+  // TODO set as (MigrationVersion.v1_3, StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME)
   val GENERATE_SAVEPOINT_VER: MigrationVersion = MigrationVersion.v1_3
-  val GENERATE_SAVEPOINT_BACKEND_TYPE: String = AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME
+  val GENERATE_SAVEPOINT_BACKEND_TYPE: String = StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME
 
   val SCALA_VERSION: String = {
     val versionString = Properties.versionString.split(" ")(1)
@@ -86,9 +86,9 @@ class StatefulJobSavepointMigrationITCase(
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
 
     StatefulJobSavepointMigrationITCase.GENERATE_SAVEPOINT_BACKEND_TYPE match {
-      case AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME =>
+      case StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME =>
         env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()))
-      case AbstractStateBackend.MEMORY_STATE_BACKEND_NAME =>
+      case StateBackendLoader.MEMORY_STATE_BACKEND_NAME =>
         env.setStateBackend(new MemoryStateBackend())
       case _ => throw new UnsupportedOperationException
     }
@@ -129,9 +129,9 @@ class StatefulJobSavepointMigrationITCase(
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
 
     migrationVersionAndBackend._2 match {
-      case AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME =>
+      case StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME =>
         env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()))
-      case AbstractStateBackend.MEMORY_STATE_BACKEND_NAME =>
+      case StateBackendLoader.MEMORY_STATE_BACKEND_NAME =>
         env.setStateBackend(new MemoryStateBackend())
       case _ => throw new UnsupportedOperationException
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/e52db8bc/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 55669f1..5a5fb74 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -20,9 +20,9 @@ package org.apache.flink.yarn;
 
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -32,7 +32,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
@@ -122,8 +121,8 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 
 		flinkYarnClient.setDynamicPropertiesEncoded("recovery.mode=zookeeper@@recovery.zookeeper.quorum=" +
 			zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts +
-			"@@" + CoreOptions.STATE_BACKEND + "=FILESYSTEM" +
-			"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
+			"@@" + CheckpointingOptions.STATE_BACKEND.key() + "=FILESYSTEM" +
+			"@@" + CheckpointingOptions.CHECKPOINTS_DIRECTORY + "=" + fsStateHandlePath + "/checkpoints" +
 			"@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery");
 
 		ClusterClient<ApplicationId> yarnCluster = null;


[13/17] flink git commit: [hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.configuration'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/ConfigDocsCompletenessChecker.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigDocsCompletenessChecker.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigDocsCompletenessChecker.java
index 741a483..3db85e7 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigDocsCompletenessChecker.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigDocsCompletenessChecker.java
@@ -31,17 +31,17 @@ import java.lang.reflect.Modifier;
 public class ConfigDocsCompletenessChecker {
 
 	public static void main(String[] args) throws Exception {
-		
+
 		String configFileContents = FileUtils.readFileToString(new File("docs/setup/config.md"));
 		Field[] fields = ConfigConstants.class.getFields();
-		
+
 		for (Field field : fields) {
 			if (Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class) &&
-					!field.getName().startsWith("DEFAULT"))
-			{
+					!field.getName().startsWith("DEFAULT")) {
+
 				Object val = field.get(null);
-				if (!configFileContents.contains((String)val)) {
-					System.out.println("++++ "+val+" is not mentioned in the configuration file!!!");
+				if (!configFileContents.contains((String) val)) {
+					System.out.println("++++ " + val + " is not mentioned in the configuration file!!!");
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/ConfigOptionsDocGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigOptionsDocGeneratorTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigOptionsDocGeneratorTest.java
index 04facf0..0959f15 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigOptionsDocGeneratorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigOptionsDocGeneratorTest.java
@@ -15,15 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
-import java.util.HashMap;
-import java.util.List;
 import org.apache.flink.api.java.tuple.Tuple2;
+
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.util.HashMap;
+import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the ConfigOptionsDocGenerator.
+ */
+@SuppressWarnings("unused")
 public class ConfigOptionsDocGeneratorTest {
 
 	static class TestConfigGroup {
@@ -67,8 +74,8 @@ public class ConfigOptionsDocGeneratorTest {
 	}
 
 	@ConfigGroups(groups = {
-			@ConfigGroup(name = "firstGroup", keyPrefix = "first"),
-			@ConfigGroup(name = "secondGroup", keyPrefix = "second")})
+	@ConfigGroup(name = "firstGroup", keyPrefix = "first"),
+	@ConfigGroup(name = "secondGroup", keyPrefix = "second")})
 	static class TestConfigMultipleSubGroup {
 		public static ConfigOption<Integer> firstOption = ConfigOptions
 			.key("first.option.a")
@@ -103,58 +110,58 @@ public class ConfigOptionsDocGeneratorTest {
 		}
 
 		assertEquals("<table class=\"table table-bordered\">" +
-		             "<thead>" +
-		             "<tr>" +
-		             "<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
-		             "<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
-		             "<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
-		             "</tr>" +
-		             "</thead>" +
-		             "<tbody>" +
-		             "<tr>" +
-		             "<td><h5>first.option.a</h5></td>" +
-		             "<td>2</td>" +
-		             "<td>This is example description for the first option.</td>" +
-		             "</tr>" +
-		             "</tbody>" +
-		             "</table>", tablesConverted.get("firstGroup"));
+				"<thead>" +
+				"<tr>" +
+				"<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
+				"<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
+				"<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
+				"</tr>" +
+				"</thead>" +
+				"<tbody>" +
+				"<tr>" +
+				"<td><h5>first.option.a</h5></td>" +
+				"<td>2</td>" +
+				"<td>This is example description for the first option.</td>" +
+				"</tr>" +
+				"</tbody>" +
+				"</table>", tablesConverted.get("firstGroup"));
 		assertEquals("<table class=\"table table-bordered\">" +
-		             "<thead>" +
-		             "<tr>" +
-		             "<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
-		             "<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
-		             "<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
-		             "</tr>" +
-		             "</thead>" +
-		             "<tbody>" +
-		             "<tr>" +
-		             "<td><h5>second.option.a</h5></td>" +
-		             "<td>(none)</td>" +
-		             "<td>This is long example description for the second option.</td>" +
-		             "</tr>" +
-		             "</tbody>" +
-		             "</table>", tablesConverted.get("secondGroup"));
+				"<thead>" +
+				"<tr>" +
+				"<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
+				"<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
+				"<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
+				"</tr>" +
+				"</thead>" +
+				"<tbody>" +
+				"<tr>" +
+				"<td><h5>second.option.a</h5></td>" +
+				"<td>(none)</td>" +
+				"<td>This is long example description for the second option.</td>" +
+				"</tr>" +
+				"</tbody>" +
+				"</table>", tablesConverted.get("secondGroup"));
 		assertEquals("<table class=\"table table-bordered\">" +
-		             "<thead>" +
-		             "<tr>" +
-		             "<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
-		             "<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
-		             "<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
-		             "</tr>" +
-		             "</thead>" +
-		             "<tbody>" +
-		             "<tr>" +
-		             "<td><h5>fourth.option.a</h5></td>" +
-		             "<td>(none)</td>" +
-		             "<td>This is long example description for the fourth option.</td>" +
-		             "</tr>" +
-		             "<tr>" +
-		             "<td><h5>third.option.a</h5></td>" +
-		             "<td>2</td>" +
-		             "<td>This is example description for the third option.</td>" +
-		             "</tr>" +
-		             "</tbody>" +
-		             "</table>", tablesConverted.get("default"));
+				"<thead>" +
+				"<tr>" +
+				"<th class=\"text-left\" style=\"width: 20%\">Key</th>" +
+				"<th class=\"text-left\" style=\"width: 15%\">Default Value</th>" +
+				"<th class=\"text-left\" style=\"width: 65%\">Description</th>" +
+				"</tr>" +
+				"</thead>" +
+				"<tbody>" +
+				"<tr>" +
+				"<td><h5>fourth.option.a</h5></td>" +
+				"<td>(none)</td>" +
+				"<td>This is long example description for the fourth option.</td>" +
+				"</tr>" +
+				"<tr>" +
+				"<td><h5>third.option.a</h5></td>" +
+				"<td>2</td>" +
+				"<td>This is example description for the third option.</td>" +
+				"</tr>" +
+				"</tbody>" +
+				"</table>", tablesConverted.get("default"));
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
index fa86fed..232c829 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
@@ -18,17 +18,17 @@
 
 package org.apache.flink.configuration;
 
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
 /**
  * This class contains test for the configuration package. In particular, the serialization of {@link Configuration}
  * objects is tested.
@@ -52,9 +52,9 @@ public class ConfigurationTest extends TestLogger {
 			orig.setFloat("PI", 3.1415926f);
 			orig.setDouble("E", Math.E);
 			orig.setBoolean("shouldbetrue", true);
-			orig.setBytes("bytes sequence", new byte[] { 1, 2, 3, 4, 5 } );
+			orig.setBytes("bytes sequence", new byte[] { 1, 2, 3, 4, 5 });
 			orig.setClass("myclass", this.getClass());
-	
+
 			final Configuration copy = InstantiationUtil.createCopyWritable(orig);
 			assertEquals("myvalue", copy.getString("mykey", "null"));
 			assertEquals(100, copy.getInteger("mynumber", 0));
@@ -64,7 +64,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(true, copy.getBoolean("shouldbetrue", false));
 			assertArrayEquals(new byte[] { 1, 2, 3, 4, 5 }, copy.getBytes("bytes sequence", null));
 			assertEquals(getClass(), copy.getClass("myclass", null, getClass().getClassLoader()));
-			
+
 			assertEquals(orig, copy);
 			assertEquals(orig.keySet(), copy.keySet());
 			assertEquals(orig.hashCode(), copy.hashCode());
@@ -79,7 +79,7 @@ public class ConfigurationTest extends TestLogger {
 	public void testConversions() {
 		try {
 			Configuration pc = new Configuration();
-			
+
 			pc.setInteger("int", 5);
 			pc.setLong("long", 15);
 			pc.setLong("too_long", TOO_LONG);
@@ -91,7 +91,7 @@ public class ConfigurationTest extends TestLogger {
 			pc.setString("string", "42");
 			pc.setString("non_convertible_string", "bcdefg&&");
 			pc.setBoolean("boolean", true);
-			
+
 			// as integer
 			assertEquals(5, pc.getInteger("int", 0));
 			assertEquals(5L, pc.getLong("int", 0));
@@ -100,7 +100,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("int", true));
 			assertEquals("5", pc.getString("int", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("int", EMPTY_BYTES));
-			
+
 			// as long
 			assertEquals(15, pc.getInteger("long", 0));
 			assertEquals(15L, pc.getLong("long", 0));
@@ -109,7 +109,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("long", true));
 			assertEquals("15", pc.getString("long", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("long", EMPTY_BYTES));
-			
+
 			// as too long
 			assertEquals(0, pc.getInteger("too_long", 0));
 			assertEquals(TOO_LONG, pc.getLong("too_long", 0));
@@ -118,7 +118,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("too_long", true));
 			assertEquals(String.valueOf(TOO_LONG), pc.getString("too_long", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long", EMPTY_BYTES));
-			
+
 			// as float
 			assertEquals(0, pc.getInteger("float", 0));
 			assertEquals(0L, pc.getLong("float", 0));
@@ -127,7 +127,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("float", true));
 			assertTrue(pc.getString("float", "0").startsWith("2.145677"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("float", EMPTY_BYTES));
-			
+
 			// as double
 			assertEquals(0, pc.getInteger("double", 0));
 			assertEquals(0L, pc.getLong("double", 0));
@@ -163,7 +163,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("too_long_double", true));
 			assertEquals(String.valueOf(TOO_LONG_DOUBLE), pc.getString("too_long_double", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long_double", EMPTY_BYTES));
-			
+
 			// as string
 			assertEquals(42, pc.getInteger("string", 0));
 			assertEquals(42L, pc.getLong("string", 0));
@@ -172,7 +172,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("string", true));
 			assertEquals("42", pc.getString("string", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("string", EMPTY_BYTES));
-			
+
 			// as non convertible string
 			assertEquals(0, pc.getInteger("non_convertible_string", 0));
 			assertEquals(0L, pc.getLong("non_convertible_string", 0));
@@ -181,7 +181,7 @@ public class ConfigurationTest extends TestLogger {
 			assertEquals(false, pc.getBoolean("non_convertible_string", true));
 			assertEquals("bcdefg&&", pc.getString("non_convertible_string", "0"));
 			assertArrayEquals(EMPTY_BYTES, pc.getBytes("non_convertible_string", EMPTY_BYTES));
-			
+
 			// as boolean
 			assertEquals(0, pc.getInteger("boolean", 0));
 			assertEquals(0L, pc.getLong("boolean", 0));
@@ -201,13 +201,13 @@ public class ConfigurationTest extends TestLogger {
 	public void testCopyConstructor() {
 		try {
 			final String key = "theKey";
-			
+
 			Configuration cfg1 = new Configuration();
 			cfg1.setString(key, "value");
-			
+
 			Configuration cfg2 = new Configuration(cfg1);
 			cfg2.setString(key, "another value");
-			
+
 			assertEquals("value", cfg1.getString(key, ""));
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
index 9298a14..cd201f1 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.configuration;
 
 import org.junit.Test;
@@ -26,10 +25,12 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-
+/**
+ * Tests for the {@link DelegatingConfiguration}.
+ */
 public class DelegatingConfigurationTest {
 
 	@Test
@@ -40,7 +41,7 @@ public class DelegatingConfigurationTest {
 		Method[] delegateMethods = DelegatingConfiguration.class.getDeclaredMethods();
 
 		for (Method configurationMethod : confMethods) {
-			if (!Modifier.isPublic(configurationMethod.getModifiers()) ) {
+			if (!Modifier.isPublic(configurationMethod.getModifiers())) {
 				continue;
 			}
 
@@ -67,11 +68,11 @@ public class DelegatingConfigurationTest {
 				}
 			}
 
-			assertTrue("Configuration method '" + configurationMethod.getName() + 
+			assertTrue("Configuration method '" + configurationMethod.getName() +
 					"' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
 		}
 	}
-	
+
 	@Test
 	public void testDelegationConfigurationWithNullPrefix() {
 		Configuration backingConf = new Configuration();
@@ -98,7 +99,6 @@ public class DelegatingConfigurationTest {
 
 		DelegatingConfiguration configuration = new DelegatingConfiguration(backingConf, prefix);
 		Set<String> keySet = configuration.keySet();
-		
 
 		assertEquals(keySet.size(), 1);
 		assertEquals(keySet.iterator().next(), expectedKey);

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java b/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java
index 1cbc8f2..05ff315 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java
@@ -37,6 +37,9 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Tests for the configuration of the default file system scheme.
+ */
 public class FilesystemSchemeConfigTest extends TestLogger {
 
 	@Rule

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java
index 322e64d..c5d2e62 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/GlobalConfigurationTest.java
@@ -18,8 +18,11 @@
 
 package org.apache.flink.configuration;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -27,10 +30,8 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.UUID;
 
-import org.apache.flink.util.TestLogger;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * This class contains tests for the global configuration (parsing configuration directory information).
@@ -112,7 +113,7 @@ public class GlobalConfigurationTest extends TestLogger {
 	public void testInvalidYamlFile() throws IOException {
 		final File confFile = tempFolder.newFile(GlobalConfiguration.FLINK_CONF_FILENAME);
 
-		try (PrintWriter pw = new PrintWriter(confFile);) {
+		try (PrintWriter pw = new PrintWriter(confFile)) {
 			pw.append("invalid");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java b/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java
index 9df541e..3fbc58b 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java
@@ -24,7 +24,9 @@ import org.junit.Test;
 
 import java.io.IOException;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 /**
  * Tests for the {@link MemorySize} class.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
index 26e3d7a..8163c50 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
@@ -18,12 +18,8 @@
 
 package org.apache.flink.configuration;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
 import java.lang.reflect.InvocationTargetException;
@@ -31,12 +27,17 @@ import java.lang.reflect.Method;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * This class verifies that the Unmodifiable Configuration class overrides all setter methods in
  * Configuration.
  */
 public class UnmodifiableConfigurationTest extends TestLogger {
-	
+
 	@Test
 	public void testOverrideAddMethods() {
 		try {
@@ -52,14 +53,14 @@ public class UnmodifiableConfigurationTest extends TestLogger {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testExceptionOnSet() {
 		try {
 			@SuppressWarnings("rawtypes")
 			final ConfigOption rawOption = ConfigOptions.key("testkey").defaultValue("value");
 
-			Map<Class<?>, Object> parameters = new HashMap<Class<?>, Object>();
+			Map<Class<?>, Object> parameters = new HashMap<>();
 			parameters.put(byte[].class, new byte[0]);
 			parameters.put(Class.class, Object.class);
 			parameters.put(int.class, 0);

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/tools/maven/suppressions-core.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions-core.xml b/tools/maven/suppressions-core.xml
index 9640815..7b0ccc7 100644
--- a/tools/maven/suppressions-core.xml
+++ b/tools/maven/suppressions-core.xml
@@ -104,14 +104,6 @@ under the License.
 		checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
 
 	<suppress
-		files="(.*)configuration[/\\](.*)"
-		checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
-	<!--Only additional checks for test sources. Those checks were present in the "pre-strict" checkstyle but were not applied to test sources. We do not want to suppress them for sources directory-->
-	<suppress
-		files="(.*)test[/\\](.*)configuration[/\\](.*)"
-		checks="AvoidStarImport"/>
-
-	<suppress
 		files="(.*)core[/\\]io[/\\](.*)"
 		checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
 	<!--Only additional checks for test sources. Those checks were present in the "pre-strict" checkstyle but were not applied to test sources. We do not want to suppress them for sources directory-->


[14/17] flink git commit: [hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.configuration'

Posted by se...@apache.org.
[hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.configuration'


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

Branch: refs/heads/master
Commit: 1f9c2d9740ffea2b59b8f5f3da287a0dc890ddbf
Parents: 212ee3d
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jan 18 17:27:13 2018 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:04 2018 +0100

----------------------------------------------------------------------
 flink-core/pom.xml                              |  21 +-
 .../apache/flink/configuration/AkkaOptions.java |   1 -
 .../flink/configuration/BlobServerOptions.java  |   2 +-
 .../flink/configuration/ConfigConstants.java    | 219 ++++++++++---------
 .../apache/flink/configuration/ConfigGroup.java |   6 +-
 .../flink/configuration/ConfigGroups.java       |   4 +-
 .../flink/configuration/ConfigOption.java       |  16 +-
 .../flink/configuration/ConfigOptions.java      |  22 +-
 .../ConfigOptionsDocGenerator.java              |   1 +
 .../flink/configuration/Configuration.java      | 117 +++++-----
 .../apache/flink/configuration/CoreOptions.java |   1 -
 .../configuration/DelegatingConfiguration.java  |   8 +-
 .../configuration/GlobalConfiguration.java      |  33 +--
 .../configuration/HeartbeatManagerOptions.java  |   6 +-
 .../configuration/HighAvailabilityOptions.java  |  28 +--
 .../configuration/HistoryServerOptions.java     |   1 +
 .../IllegalConfigurationException.java          |   2 +-
 .../flink/configuration/JobManagerOptions.java  |  14 +-
 .../apache/flink/configuration/MemorySize.java  |  28 +--
 .../flink/configuration/MetricOptions.java      |   6 +-
 .../configuration/ResourceManagerOptions.java   |   6 +-
 .../apache/flink/configuration/RestOptions.java |   1 +
 .../flink/configuration/TaskManagerOptions.java |  14 +-
 .../UnmodifiableConfiguration.java              |   5 +-
 .../apache/flink/configuration/WebOptions.java  |   9 +-
 .../ConfigDocsCompletenessChecker.java          |  12 +-
 .../ConfigOptionsDocGeneratorTest.java          | 117 +++++-----
 .../flink/configuration/ConfigurationTest.java  |  40 ++--
 .../DelegatingConfigurationTest.java            |  14 +-
 .../FilesystemSchemeConfigTest.java             |   3 +
 .../configuration/GlobalConfigurationTest.java  |  15 +-
 .../flink/configuration/MemorySizeTest.java     |   4 +-
 .../UnmodifiableConfigurationTest.java          |  17 +-
 tools/maven/suppressions-core.xml               |   8 -
 34 files changed, 421 insertions(+), 380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index 0465227..285074e 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -204,7 +204,7 @@ under the License.
 							<exclude>org.apache.flink.api.java.typeutils.AvroTypeInfo</exclude>
 							<!-- Breaking changes between 1.1 and 1.2.
 							We ignore these changes because these are low-level, internal runtime configuration parameters -->
-							<exclude>org.apache.flink.configuration.ConfigConstants#DEFAULT_JOB_MANAGER_MAX_ATTEMPTS_HISTORY_SIZE</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#DEFAULT_NETWORK_REQUEST_BACKOFF_INITIAL</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#DEFAULT_NETWORK_REQUEST_BACKOFF_MAX</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#DEFAULT_TASK_CANCELLATION_TIMEOUT_MILLIS</exclude>
@@ -212,6 +212,25 @@ under the License.
 							<exclude>org.apache.flink.configuration.ConfigConstants#NETWORK_REQUEST_BACKOFF_INITIAL_KEY</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#NETWORK_REQUEST_BACKOFF_MAX_KEY</exclude>
 
+							<!-- fields that were accidentally not final in the beginning -->
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_WATCH_THRESHOLD</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_DISPATCHER_THROUGHPUT</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_SECURITY_SSL_VERIFY_HOSTNAME</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVAL</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_RECOVERY_MODE</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_SECURITY_SSL_ENABLED</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_CLIENT_TIMEOUT</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_STATE_BACKEND</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_TRANSPORT_THRESHOLD</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_FRAMESIZE</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_SECURITY_SSL_PROTOCOL</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_LOG_LIFECYCLE_EVENTS</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_SECURITY_SSL_ALGORITHMS</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_TRANSPORT_HEARTBEAT_PAUSE</exclude>
+							<exclude>org.apache.flink.configuration.ConfigConstants.DEFAULT_AKKA_SSL_ENABLED</exclude>
+
 							<!-- apparently there is a bug in the plugin which makes it fail on this new file, event though
 								its new, and not conflicting/breaking -->
 							<exclude>org.apache.flink.api.common.serialization.DeserializationSchema</exclude>

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java
index cf65939..09ea490 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java
@@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving;
 
 /**
  * Akka configuration options.
- *
  * TODO: Migrate other akka config options to this file
  */
 @PublicEvolving

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
index 8680096..a813d01 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
@@ -62,7 +62,7 @@ public class BlobServerOptions {
 	 * a range of ports: "50100-50200"
 	 * or a list of ranges and or points: "50100-50200,50300-50400,51234"
 	 *
-	 * Setting the port to 0 will let the OS choose an available port.
+	 * <p>Setting the port to 0 will let the OS choose an available port.
 	 */
 	public static final ConfigOption<String> PORT =
 		key("blob.server.port")

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index d1dcef5..7b76f53 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -31,12 +31,13 @@ import static org.apache.flink.configuration.ConfigOptions.key;
  * the default values.
  */
 @Public
+@SuppressWarnings("unused")
 public final class ConfigConstants {
 
 	// ------------------------------------------------------------------------
 	//                            Configuration Keys
 	// ------------------------------------------------------------------------
-	
+
 	// ---------------------------- Parallelism -------------------------------
 
 	/**
@@ -112,7 +113,7 @@ public final class ConfigConstants {
 	@Deprecated
 	@PublicEvolving
 	public static final String EXECUTION_RETRY_DELAY_KEY = "execution-retries.delay";
-	
+
 	// -------------------------------- Runtime -------------------------------
 
 	/**
@@ -202,7 +203,7 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_DATA_PORT_KEY = "taskmanager.data.port";
 
 	/**
-	 * Config parameter to override SSL support for taskmanager's data transport
+	 * Config parameter to override SSL support for taskmanager's data transport.
 	 */
 	public static final String TASK_MANAGER_DATA_SSL_ENABLED = "taskmanager.data.ssl.enabled";
 
@@ -216,7 +217,7 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_TMP_DIR_KEY = "taskmanager.tmp.dirs";
 
 	/**
-	 * The config parameter defining the taskmanager log file location
+	 * The config parameter defining the taskmanager log file location.
 	 */
 	public static final String TASK_MANAGER_LOG_PATH_KEY = "taskmanager.log.path";
 
@@ -229,7 +230,7 @@ public final class ConfigConstants {
 	 */
 	@Deprecated
 	public static final String TASK_MANAGER_MEMORY_SIZE_KEY = "taskmanager.memory.size";
-	
+
 	/**
 	 * The config parameter defining the fraction of free memory allocated by the memory manager.
 	 *
@@ -271,7 +272,7 @@ public final class ConfigConstants {
 	 */
 	@Deprecated
 	public static final String TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY = "taskmanager.memory.segment-size";
-	
+
 	/**
 	 * The implementation to use for spillable/spilled intermediate results, which have both
 	 * synchronous and asynchronous implementations: "sync" or "async".
@@ -323,14 +324,14 @@ public final class ConfigConstants {
 	public static final String TASK_CANCELLATION_INTERVAL_MILLIS = "task.cancellation-interval";
 
 	// --------------------------- Runtime Algorithms -------------------------------
-	
+
 	/**
 	 * Parameter for the maximum fan for out-of-core algorithms.
 	 * Corresponds to the maximum fan-in for merge-sorts and the maximum fan-out
-	 * for hybrid hash joins. 
+	 * for hybrid hash joins.
 	 */
 	public static final String DEFAULT_SPILLING_MAX_FAN_KEY = "taskmanager.runtime.max-fan";
-	
+
 	/**
 	 * Key for the default spilling threshold. When more than the threshold memory of the sort buffers is full, the
 	 * sorter will start spilling to disk.
@@ -341,7 +342,7 @@ public final class ConfigConstants {
 	 * Parameter to switch hash join bloom filters for spilled partitions on and off.
 	 */
 	public static final String RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEY = "taskmanager.runtime.hashjoin-bloom-filters";
-	
+
 	/**
 	 * The config parameter defining the timeout for filesystem stream opening.
 	 * A value of 0 indicates infinite waiting.
@@ -383,13 +384,13 @@ public final class ConfigConstants {
 
 	/**
 	 * Similar to the {@see CONTAINERIZED_MASTER_ENV_PREFIX}, this configuration prefix allows
-	 * setting custom environment variables for the workers (TaskManagers)
+	 * setting custom environment variables for the workers (TaskManagers).
 	 * @deprecated Use {@link ResourceManagerOptions#CONTAINERIZED_TASK_MANAGER_ENV_PREFIX} instead.
 	 */
 	@Deprecated
 	public static final String CONTAINERIZED_TASK_MANAGER_ENV_PREFIX = "containerized.taskmanager.env.";
 
-	
+
 	// ------------------------ YARN Configuration ------------------------
 
 	/**
@@ -425,7 +426,8 @@ public final class ConfigConstants {
 	 * The maximum number of failed YARN containers before entirely stopping
 	 * the YARN session / job on YARN.
 	 *
-	 * By default, we take the number of initially requested containers.
+	 * <p>By default, we take the number of initially requested containers.
+	 *
 	 * @deprecated in favor of {@code YarnConfigOptions#MAX_FAILED_CONTAINERS}.
 	 */
 	@Deprecated
@@ -435,7 +437,8 @@ public final class ConfigConstants {
 	 * Set the number of retries for failed YARN ApplicationMasters/JobManagers in high
 	 * availability mode. This value is usually limited by YARN.
 	 *
-	 * By default, it's 1 in the standalone case and 2 in the high availability case.
+	 * <p>By default, it's 1 in the standalone case and 2 in the high availability case.
+	 *
 	 * @deprecated in favor of {@code YarnConfigOptions#APPLICATION_ATTEMPTS}.
 	 */
 	@Deprecated
@@ -443,7 +446,6 @@ public final class ConfigConstants {
 
 	/**
 	 * The heartbeat interval between the Application Master and the YARN Resource Manager.
-	 *
 	 * The default value is 5 (seconds).
 	 * @deprecated in favor of {@code YarnConfigOptions#HEARTBEAT_DELAY_SECONDS}.
 	 */
@@ -492,16 +494,17 @@ public final class ConfigConstants {
 	 */
 	public static final String YARN_CONTAINER_START_COMMAND_TEMPLATE =
 		"yarn.container-start-command-template";
-	
+
 	 /**
 	 * The config parameter defining the Akka actor system port for the ApplicationMaster and
 	 * JobManager
 	 *
-	 * The port can either be a port, such as "9123",
+	 * <p>The port can either be a port, such as "9123",
 	 * a range of ports: "50100-50200"
 	 * or a list of ranges and or points: "50100-50200,50300-50400,51234"
 	 *
-	 * Setting the port to 0 will let the OS choose an available port.
+	 * <p>Setting the port to 0 will let the OS choose an available port.
+	 *
 	 * @deprecated in favor of {@code YarnConfigOptions#APPLICATION_MASTER_PORT}.
 	 */
 	@Deprecated
@@ -528,7 +531,8 @@ public final class ConfigConstants {
 	 * The maximum number of failed Mesos tasks before entirely stopping
 	 * the Mesos session / job on Mesos.
 	 *
-	 * By default, we take the number of initially requested tasks.
+	 * <p>By default, we take the number of initially requested tasks.
+	 *
 	 * @deprecated in favor of {@code MesosOptions#MAX_FAILED_TASKS}.
 	 */
 	@Deprecated
@@ -537,7 +541,7 @@ public final class ConfigConstants {
 	/**
 	 * The Mesos master URL.
 	 *
-	 * The value should be in one of the following forms:
+	 * <p>The value should be in one of the following forms:
 	 * <pre>
 	 * {@code
 	 *     host:port
@@ -554,7 +558,8 @@ public final class ConfigConstants {
 	/**
 	 * The failover timeout for the Mesos scheduler, after which running tasks are automatically shut down.
 	 *
-	 * The default value is 600 (seconds).
+	 * <p>The default value is 600 (seconds).
+	 *
 	 * @deprecated in favor of {@code MesosOptions#FAILOVER_TIMEOUT_SECONDS}.
 	 */
 	@Deprecated
@@ -589,7 +594,7 @@ public final class ConfigConstants {
 	public static final String MESOS_RESOURCEMANAGER_FRAMEWORK_USER = "mesos.resourcemanager.framework.user";
 
 	/**
-	 * Config parameter to override SSL support for the Artifact Server
+	 * Config parameter to override SSL support for the Artifact Server.
 	 * @deprecated in favor of {@code MesosOptions#ARTIFACT_SERVER_SSL_ENABLED}.
 	 */
 	@Deprecated
@@ -598,23 +603,23 @@ public final class ConfigConstants {
 	// ------------------------ Hadoop Configuration ------------------------
 
 	/**
-	 * Path to hdfs-default.xml file
+	 * Path to hdfs-default.xml file.
 	 *
 	 * @deprecated Use environment variable HADOOP_CONF_DIR instead.
 	 */
 	@Deprecated
 	public static final String HDFS_DEFAULT_CONFIG = "fs.hdfs.hdfsdefault";
-	
+
 	/**
-	 * Path to hdfs-site.xml file
+	 * Path to hdfs-site.xml file.
 	 *
 	 * @deprecated Use environment variable HADOOP_CONF_DIR instead.
 	 */
 	@Deprecated
 	public static final String HDFS_SITE_CONFIG = "fs.hdfs.hdfssite";
-	
+
 	/**
-	 * Path to Hadoop configuration
+	 * Path to Hadoop configuration.
 	 *
 	 * @deprecated Use environment variable HADOOP_CONF_DIR instead.
 	 */
@@ -661,8 +666,8 @@ public final class ConfigConstants {
 	 * The maximum length of a single sampled record before the sampling is aborted.
 	 */
 	public static final String DELIMITED_FORMAT_MAX_SAMPLE_LENGTH_KEY = "compiler.delimited-informat.max-sample-len";
-	
-	
+
+
 	// ------------------------- JobManager Web Frontend ----------------------
 
 	/**
@@ -674,7 +679,7 @@ public final class ConfigConstants {
 	public static final String JOB_MANAGER_WEB_PORT_KEY = "jobmanager.web.port";
 
 	/**
-	 * Config parameter to override SSL support for the JobManager Web UI
+	 * Config parameter to override SSL support for the JobManager Web UI.
 	 *
 	 * @deprecated Use {@link WebOptions#SSL_ENABLED} instead.
 	 */
@@ -699,7 +704,7 @@ public final class ConfigConstants {
 	public static final String JOB_MANAGER_WEB_UPLOAD_DIR_KEY = "jobmanager.web.upload.dir";
 
 	/**
-	 * The config parameter defining the number of archived jobs for the jobmanager
+	 * The config parameter defining the number of archived jobs for the jobmanager.
 	 *
 	 * @deprecated Use {@link WebOptions#ARCHIVE_COUNT} instead.
 	 */
@@ -707,7 +712,7 @@ public final class ConfigConstants {
 	public static final String JOB_MANAGER_WEB_ARCHIVE_COUNT = "jobmanager.web.history";
 
 	/**
-	 * The log file location (may be in /log for standalone but under log directory when using YARN)
+	 * The log file location (may be in /log for standalone but under log directory when using YARN).
 	 *
 	 * @deprecated Use {@link WebOptions#LOG_PATH} instead.
 	 */
@@ -773,7 +778,7 @@ public final class ConfigConstants {
 	// ------------------------------ AKKA ------------------------------------
 
 	/**
-	 * Timeout for the startup of the actor system
+	 * Timeout for the startup of the actor system.
 	 *
 	 * @deprecated Use {@link AkkaOptions#STARTUP_TIMEOUT} instead.
 	 */
@@ -781,7 +786,7 @@ public final class ConfigConstants {
 	public static final String AKKA_STARTUP_TIMEOUT = "akka.startup-timeout";
 
 	/**
-	 * Heartbeat interval of the transport failure detector
+	 * Heartbeat interval of the transport failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_HEARTBEAT_INTERVAL} instead.
 	 */
@@ -789,7 +794,7 @@ public final class ConfigConstants {
 	public static final String AKKA_TRANSPORT_HEARTBEAT_INTERVAL = "akka.transport.heartbeat.interval";
 
 	/**
-	 * Allowed heartbeat pause for the transport failure detector
+	 * Allowed heartbeat pause for the transport failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_HEARTBEAT_PAUSE} instead.
 	 */
@@ -797,7 +802,7 @@ public final class ConfigConstants {
 	public static final String AKKA_TRANSPORT_HEARTBEAT_PAUSE = "akka.transport.heartbeat.pause";
 
 	/**
-	 * Detection threshold of transport failure detector
+	 * Detection threshold of transport failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_THRESHOLD} instead.
 	 */
@@ -805,7 +810,7 @@ public final class ConfigConstants {
 	public static final String AKKA_TRANSPORT_THRESHOLD = "akka.transport.threshold";
 
 	/**
-	 * Heartbeat interval of watch failure detector
+	 * Heartbeat interval of watch failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#WATCH_HEARTBEAT_INTERVAL} instead.
 	 */
@@ -813,7 +818,7 @@ public final class ConfigConstants {
 	public static final String AKKA_WATCH_HEARTBEAT_INTERVAL = "akka.watch.heartbeat.interval";
 
 	/**
-	 * Allowed heartbeat pause for the watch failure detector
+	 * Allowed heartbeat pause for the watch failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#WATCH_HEARTBEAT_PAUSE} instead.
 	 */
@@ -821,7 +826,7 @@ public final class ConfigConstants {
 	public static final String AKKA_WATCH_HEARTBEAT_PAUSE = "akka.watch.heartbeat.pause";
 
 	/**
-	 * Detection threshold for the phi accrual watch failure detector
+	 * Detection threshold for the phi accrual watch failure detector.
 	 *
 	 * @deprecated Use {@link AkkaOptions#WATCH_THRESHOLD} instead.
 	 */
@@ -829,7 +834,7 @@ public final class ConfigConstants {
 	public static final String AKKA_WATCH_THRESHOLD = "akka.watch.threshold";
 
 	/**
-	 * Akka TCP timeout
+	 * Akka TCP timeout.
 	 *
 	 * @deprecated Use {@link AkkaOptions#TCP_TIMEOUT} instead.
 	 */
@@ -837,7 +842,7 @@ public final class ConfigConstants {
 	public static final String AKKA_TCP_TIMEOUT = "akka.tcp.timeout";
 
 	/**
-	 * Override SSL support for the Akka transport
+	 * Override SSL support for the Akka transport.
 	 *
 	 * @deprecated Use {@link AkkaOptions#SSL_ENABLED} instead.
 	 */
@@ -845,7 +850,7 @@ public final class ConfigConstants {
 	public static final String AKKA_SSL_ENABLED = "akka.ssl.enabled";
 
 	/**
-	 * Maximum framesize of akka messages
+	 * Maximum framesize of akka messages.
 	 *
 	 * @deprecated Use {@link AkkaOptions#FRAMESIZE} instead.
 	 */
@@ -853,7 +858,7 @@ public final class ConfigConstants {
 	public static final String AKKA_FRAMESIZE = "akka.framesize";
 
 	/**
-	 * Maximum number of messages until another actor is executed by the same thread
+	 * Maximum number of messages until another actor is executed by the same thread.
 	 *
 	 * @deprecated Use {@link AkkaOptions#DISPATCHER_THROUGHPUT} instead.
 	 */
@@ -861,7 +866,7 @@ public final class ConfigConstants {
 	public static final String AKKA_DISPATCHER_THROUGHPUT = "akka.throughput";
 
 	/**
-	 * Log lifecycle events
+	 * Log lifecycle events.
 	 *
 	 * @deprecated Use {@link AkkaOptions#LOG_LIFECYCLE_EVENTS} instead.
 	 */
@@ -869,7 +874,7 @@ public final class ConfigConstants {
 	public static final String AKKA_LOG_LIFECYCLE_EVENTS = "akka.log.lifecycle.events";
 
 	/**
-	 * Timeout for all blocking calls on the cluster side
+	 * Timeout for all blocking calls on the cluster side.
 	 *
 	 * @deprecated Use {@link AkkaOptions#ASK_TIMEOUT} instead.
 	 */
@@ -877,7 +882,7 @@ public final class ConfigConstants {
 	public static final String AKKA_ASK_TIMEOUT = "akka.ask.timeout";
 
 	/**
-	 * Timeout for all blocking calls that look up remote actors
+	 * Timeout for all blocking calls that look up remote actors.
 	 *
 	 * @deprecated Use {@link AkkaOptions#LOOKUP_TIMEOUT} instead.
 	 */
@@ -885,7 +890,7 @@ public final class ConfigConstants {
 	public static final String AKKA_LOOKUP_TIMEOUT = "akka.lookup.timeout";
 
 	/**
-	 * Timeout for all blocking calls on the client side
+	 * Timeout for all blocking calls on the client side.
 	 *
 	 * @deprecated Use {@link AkkaOptions#CLIENT_TIMEOUT} instead.
 	 */
@@ -893,13 +898,13 @@ public final class ConfigConstants {
 	public static final String AKKA_CLIENT_TIMEOUT = "akka.client.timeout";
 
 	/**
-	 * Exit JVM on fatal Akka errors
+	 * Exit JVM on fatal Akka errors.
 	 *
 	 * @deprecated Use {@link AkkaOptions#JVM_EXIT_ON_FATAL_ERROR} instead.
 	 */
 	@Deprecated
 	public static final String AKKA_JVM_EXIT_ON_FATAL_ERROR = "akka.jvm-exit-on-fatal-error";
-	
+
 	// ----------------------------- Transport SSL Settings--------------------
 
 	/**
@@ -957,21 +962,21 @@ public final class ConfigConstants {
 	public static final String SECURITY_SSL_VERIFY_HOSTNAME = "security.ssl.verify-hostname";
 
 	// ----------------------------- Streaming --------------------------------
-	
+
 	/**
-	 * State backend for checkpoints
-	 * 
+	 * State backend for checkpoints.
+	 *
 	 * @deprecated Use {@link CheckpointingOptions#STATE_BACKEND} instead.
 	 */
 	@Deprecated
 	public static final String STATE_BACKEND = "state.backend";
-	
+
 	// ----------------------------- Miscellaneous ----------------------------
-	
+
 	/**
 	 * The key to the Flink base directory path. Was initially used for configurations of the
 	 * web UI, but outdated now.
-	 * 
+	 *
 	 * @deprecated This parameter should not be used any more. A running Flink cluster should
 	 *             make no assumption about its location.
 	 */
@@ -984,15 +989,15 @@ public final class ConfigConstants {
 
 	// --------------------------- High Availability --------------------------
 
-	/** Defines high availability mode used for the cluster execution ("NONE", "ZOOKEEPER") */
+	/** Defines high availability mode used for the cluster execution ("NONE", "ZOOKEEPER"). */
 	@PublicEvolving
 	public static final String HA_MODE = "high-availability";
 
-	/** Ports used by the job manager if not in 'none' recovery mode */
+	/** Ports used by the job manager if not in 'none' recovery mode. */
 	@PublicEvolving
 	public static final String HA_JOB_MANAGER_PORT = "high-availability.jobmanager.port";
 
-	/** The time before the JobManager recovers persisted jobs */
+	/** The time before the JobManager recovers persisted jobs. */
 	@PublicEvolving
 	public static final String HA_JOB_DELAY = "high-availability.job.delay";
 
@@ -1187,7 +1192,7 @@ public final class ConfigConstants {
 
 	/** The class of the reporter to use. This is used as a suffix in an actual reporter config */
 	public static final String METRICS_REPORTER_CLASS_SUFFIX = "class";
-	
+
 	/** The interval between reports. This is used as a suffix in an actual reporter config */
 	public static final String METRICS_REPORTER_INTERVAL_SUFFIX = "interval";
 
@@ -1258,12 +1263,12 @@ public final class ConfigConstants {
 	// ------------------------------------------------------------------------
 
 	// ---------------------------- Parallelism -------------------------------
-	
+
 	/**
 	 * The default parallelism for operations.
 	 */
 	public static final int DEFAULT_PARALLELISM = 1;
-	
+
 	/**
 	 * The default number of execution retries.
 	 */
@@ -1272,14 +1277,14 @@ public final class ConfigConstants {
 	// ------------------------------ Runtime ---------------------------------
 
 	/**
-	 * The default library cache manager cleanup interval in seconds
+	 * The default library cache manager cleanup interval in seconds.
 	 *
 	 * @deprecated use {@link BlobServerOptions#CLEANUP_INTERVAL} instead
 	 */
 	@Deprecated
 	public static final long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL =
 		BlobServerOptions.CLEANUP_INTERVAL.defaultValue();
-	
+
 	/**
 	 * The default network port to connect to for communication with the job manager.
 	 */
@@ -1335,7 +1340,7 @@ public final class ConfigConstants {
 	public static final int DEFAULT_TASK_MANAGER_DATA_PORT = 0;
 
 	/**
-	 * The default value to override ssl support for task manager's data transport
+	 * The default value to override ssl support for task manager's data transport.
 	 */
 	public static final boolean DEFAULT_TASK_MANAGER_DATA_SSL_ENABLED = true;
 
@@ -1388,7 +1393,7 @@ public final class ConfigConstants {
 	public static final long DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS = 5000L;
 
 	/**
-	 * The default task manager's maximum registration duration
+	 * The default task manager's maximum registration duration.
 	 */
 	public static final String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION = "Inf";
 
@@ -1420,22 +1425,22 @@ public final class ConfigConstants {
 	public static final long DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS = 30000;
 
 	// ------------------------ Runtime Algorithms ------------------------
-	
+
 	/**
 	 * Default setting for the switch for hash join bloom filters for spilled partitions.
 	 */
 	public static final boolean DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERS = false;
-	
+
 	/**
 	 * The default value for the maximum spilling fan in/out.
 	 */
 	public static final int DEFAULT_SPILLING_MAX_FAN = 128;
-	
+
 	/**
 	 * The default percentage of the sort memory to be full before data is spilled.
 	 */
 	public static final float DEFAULT_SORT_SPILLING_THRESHOLD = 0.8f;
-	
+
 	/**
 	 * The default timeout for filesystem stream opening: infinite (means max long milliseconds).
 	 */
@@ -1466,7 +1471,7 @@ public final class ConfigConstants {
 	public static final float DEFAULT_YARN_HEAP_CUTOFF_RATIO = 0.25f;
 
 	/**
-	 * Start command template for Flink on YARN containers
+	 * Start command template for Flink on YARN containers.
 	 */
 	public static final String DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE =
 		"%java% %jvmmem% %jvmopts% %logging% %class% %args% %redirects%";
@@ -1483,7 +1488,7 @@ public final class ConfigConstants {
 	// For more configuration entries please see {@code MesosTaskManagerParameters}.
 
 	/**
-	 * The default failover timeout provided to Mesos (10 mins)
+	 * The default failover timeout provided to Mesos (10 mins).
 	 * @deprecated in favor of {@code MesosOptions#FAILOVER_TIMEOUT_SECONDS}.
 	 */
 	@Deprecated
@@ -1522,12 +1527,12 @@ public final class ConfigConstants {
 
 	/**
 	 * The default filesystem to be used, if no other scheme is specified in the
-	 * user-provided URI (= local filesystem)
-	 * */
+	 * user-provided URI (= local filesystem).
+	 */
 	public static final String DEFAULT_FILESYSTEM_SCHEME = "file:///";
-	
+
 	/**
-	 * The default behavior with respect to overwriting existing files (= not overwrite)
+	 * The default behavior with respect to overwriting existing files (= not overwrite).
 	 */
 	public static final boolean DEFAULT_FILESYSTEM_OVERWRITE = false;
 
@@ -1535,26 +1540,26 @@ public final class ConfigConstants {
 	 * The default behavior for output directory creating (create only directory when parallelism &gt; 1).
 	 */
 	public static final boolean DEFAULT_FILESYSTEM_ALWAYS_CREATE_DIRECTORY = false;
-	
-	
+
+
 	// ---------------------------- Compiler -------------------------------
 
 	/**
 	 * The default maximum number of line samples taken by the delimited input format.
 	 */
 	public static final int DEFAULT_DELIMITED_FORMAT_MAX_LINE_SAMPLES = 10;
-	
+
 	/**
 	 * The default minimum number of line samples taken by the delimited input format.
 	 */
 	public static final int DEFAULT_DELIMITED_FORMAT_MIN_LINE_SAMPLES = 2;
-	
+
 	/**
 	 * The default maximum sample length before sampling is aborted (2 MiBytes).
 	 */
 	public static final int DEFAULT_DELIMITED_FORMAT_MAX_SAMPLE_LEN = 2 * 1024 * 1024;
-	
-	
+
+
 	// ------------------------- JobManager Web Frontend ----------------------
 
 	/**
@@ -1577,7 +1582,7 @@ public final class ConfigConstants {
 	public static final int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT = 8081;
 
 	/**
-	 * Default value to override SSL support for the JobManager web UI
+	 * Default value to override SSL support for the JobManager web UI.
 	 *
 	 * @deprecated use {@link WebOptions#SSL_ENABLED} instead
 	 */
@@ -1585,7 +1590,7 @@ public final class ConfigConstants {
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED = true;
 
 	/**
-	 * The default number of archived jobs for the jobmanager
+	 * The default number of archived jobs for the jobmanager.
 	 *
 	 * @deprecated use {@link WebOptions#ARCHIVE_COUNT} instead
 	 */
@@ -1650,67 +1655,67 @@ public final class ConfigConstants {
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_HEARTBEAT_INTERVAL} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVAL = "1000 s";
+	public static final String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVAL = "1000 s";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_HEARTBEAT_PAUSE} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_PAUSE = "6000 s";
+	public static final String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_PAUSE = "6000 s";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#TRANSPORT_THRESHOLD} instead.
 	 */
 	@Deprecated
-	public static double DEFAULT_AKKA_TRANSPORT_THRESHOLD = 300.0;
+	public static final double DEFAULT_AKKA_TRANSPORT_THRESHOLD = 300.0;
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#WATCH_THRESHOLD} instead.
 	 */
 	@Deprecated
-	public static double DEFAULT_AKKA_WATCH_THRESHOLD = 12;
+	public static final double DEFAULT_AKKA_WATCH_THRESHOLD = 12;
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#DISPATCHER_THROUGHPUT} instead.
 	 */
 	@Deprecated
-	public static int DEFAULT_AKKA_DISPATCHER_THROUGHPUT = 15;
+	public static final int DEFAULT_AKKA_DISPATCHER_THROUGHPUT = 15;
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#LOG_LIFECYCLE_EVENTS} instead.
 	 */
 	@Deprecated
-	public static boolean DEFAULT_AKKA_LOG_LIFECYCLE_EVENTS = false;
+	public static final boolean DEFAULT_AKKA_LOG_LIFECYCLE_EVENTS = false;
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#FRAMESIZE} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_FRAMESIZE = "10485760b";
+	public static final String DEFAULT_AKKA_FRAMESIZE = "10485760b";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#ASK_TIMEOUT} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_ASK_TIMEOUT = "10 s";
+	public static final String DEFAULT_AKKA_ASK_TIMEOUT = "10 s";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#LOOKUP_TIMEOUT} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_LOOKUP_TIMEOUT = "10 s";
+	public static final String DEFAULT_AKKA_LOOKUP_TIMEOUT = "10 s";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#CLIENT_TIMEOUT} instead.
 	 */
 	@Deprecated
-	public static String DEFAULT_AKKA_CLIENT_TIMEOUT = "60 s";
+	public static final String DEFAULT_AKKA_CLIENT_TIMEOUT = "60 s";
 
 	/**
 	 * @deprecated Use {@link AkkaOptions#SSL_ENABLED} instead.
 	 */
 	@Deprecated
-	public static boolean DEFAULT_AKKA_SSL_ENABLED = true;
+	public static final boolean DEFAULT_AKKA_SSL_ENABLED = true;
 
 	// ----------------------------- SSL Values --------------------------------
 
@@ -1718,34 +1723,34 @@ public final class ConfigConstants {
 	 * @deprecated use {@link SecurityOptions#SSL_ENABLED} instead
 	 */
 	@Deprecated
-	public static boolean DEFAULT_SECURITY_SSL_ENABLED = false;
+	public static final boolean DEFAULT_SECURITY_SSL_ENABLED = false;
 
 	/**
 	 * @deprecated use {@link SecurityOptions#SSL_PROTOCOL} instead
 	 */
 	@Deprecated
-	public static String DEFAULT_SECURITY_SSL_PROTOCOL = "TLSv1.2";
+	public static final String DEFAULT_SECURITY_SSL_PROTOCOL = "TLSv1.2";
 
 	/**
 	 * @deprecated use {@link SecurityOptions#SSL_ALGORITHMS} instead
 	 */
 	@Deprecated
-	public static String DEFAULT_SECURITY_SSL_ALGORITHMS = "TLS_RSA_WITH_AES_128_CBC_SHA";
+	public static final String DEFAULT_SECURITY_SSL_ALGORITHMS = "TLS_RSA_WITH_AES_128_CBC_SHA";
 
 	/**
 	 * @deprecated use {@link SecurityOptions#SSL_VERIFY_HOSTNAME} instead
 	 */
 	@Deprecated
-	public static boolean DEFAULT_SECURITY_SSL_VERIFY_HOSTNAME = true;
+	public static final boolean DEFAULT_SECURITY_SSL_VERIFY_HOSTNAME = true;
 
 	// ----------------------------- Streaming Values --------------------------
-	
-	public static String DEFAULT_STATE_BACKEND = "jobmanager";
+
+	public static final String DEFAULT_STATE_BACKEND = "jobmanager";
 
 	// ----------------------------- LocalExecution ----------------------------
 
 	/**
-	 * Sets the number of local task managers
+	 * Sets the number of local task managers.
 	 */
 	public static final String LOCAL_NUMBER_TASK_MANAGER = "local.number-taskmanager";
 
@@ -1772,11 +1777,11 @@ public final class ConfigConstants {
 	// --------------------------- High Availability ---------------------------------
 
 	@PublicEvolving
-	public static String DEFAULT_HA_MODE = "none";
+	public static final String DEFAULT_HA_MODE = "none";
 
 	/** @deprecated Deprecated in favour of {@link #DEFAULT_HA_MODE} */
 	@Deprecated
-	public static String DEFAULT_RECOVERY_MODE = "standalone";
+	public static final String DEFAULT_RECOVERY_MODE = "standalone";
 
 	/**
 	 * Default port used by the job manager if not in standalone recovery mode. If <code>0</code>
@@ -1898,16 +1903,16 @@ public final class ConfigConstants {
 
 	// ----------------------------- Environment Variables ----------------------------
 
-	/** The environment variable name which contains the location of the configuration directory */
+	/** The environment variable name which contains the location of the configuration directory. */
 	public static final String ENV_FLINK_CONF_DIR = "FLINK_CONF_DIR";
 
-	/** The environment variable name which contains the location of the lib folder */
+	/** The environment variable name which contains the location of the lib folder. */
 	public static final String ENV_FLINK_LIB_DIR = "FLINK_LIB_DIR";
 
-	/** The environment variable name which contains the location of the bin directory */
+	/** The environment variable name which contains the location of the bin directory. */
 	public static final String ENV_FLINK_BIN_DIR = "FLINK_BIN_DIR";
 
-	/** The environment variable name which contains the Flink installation root directory */
+	/** The environment variable name which contains the Flink installation root directory. */
 	public static final String ENV_FLINK_HOME_DIR = "FLINK_HOME";
 
 	// ---------------------------- Encoding ------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroup.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroup.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroup.java
index a0ad000..3cd1d7f 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroup.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroup.java
@@ -15,15 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
-import java.lang.annotation.Target;
 import org.apache.flink.annotation.Internal;
 
+import java.lang.annotation.Target;
+
 /**
  * A class that specifies a group of {@link ConfigOption}. The name of the group will be used as the basis for the
  * filename of the generated html file, as defined in {@link ConfigOptionsDocGenerator}.
- * 
+ *
  * @see ConfigGroups
  */
 @Target({})

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroups.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroups.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroups.java
index 2c1f871..94e52ee 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroups.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigGroups.java
@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
+import org.apache.flink.annotation.Internal;
+
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
-import org.apache.flink.annotation.Internal;
 
 /**
  * Annotation used on classes containing {@link ConfigOption}s that enables the separation of options into different

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
index 22d2cc4..8e7d79b 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
@@ -29,10 +29,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * A {@code ConfigOption} describes a configuration parameter. It encapsulates
  * the configuration key, deprecated older versions of the key, and an optional
  * default value for the configuration parameter.
- * 
+ *
  * <p>{@code ConfigOptions} are built via the {@link ConfigOptions} class.
  * Once created, a config option is immutable.
- * 
+ *
  * @param <T> The type of value associated with the configuration option.
  */
 @PublicEvolving
@@ -42,16 +42,16 @@ public class ConfigOption<T> {
 
 	// ------------------------------------------------------------------------
 
-	/** The current key for that config option */
+	/** The current key for that config option. */
 	private final String key;
 
-	/** The list of deprecated keys, in the order to be checked */
+	/** The list of deprecated keys, in the order to be checked. */
 	private final String[] deprecatedKeys;
 
-	/** The default value for this option */
+	/** The default value for this option. */
 	private final T defaultValue;
 
-	/** The description for this option */
+	/** The description for this option. */
 	private final String description;
 
 	// ------------------------------------------------------------------------
@@ -88,11 +88,11 @@ public class ConfigOption<T> {
 	/**
 	 * Creates a new config option, using this option's key and default value, and
 	 * adding the given deprecated keys.
-	 * 
+	 *
 	 * <p>When obtaining a value from the configuration via {@link Configuration#getValue(ConfigOption)},
 	 * the deprecated keys will be checked in the order provided to this method. The first key for which
 	 * a value is found will be used - that value will be returned.
-	 * 
+	 *
 	 * @param deprecatedKeys The deprecated keys, in the order in which they should be checked.
 	 * @return A new config options, with the given deprecated keys.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
index f87da0a..1ec5b3c 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
@@ -25,23 +25,23 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 /**
  * {@code ConfigOptions} are used to build a {@link ConfigOption}.
  * The option is typically built in one of the following pattern:
- * 
+ *
  * <pre>{@code
  * // simple string-valued option with a default value
  * ConfigOption<String> tempDirs = ConfigOptions
  *     .key("tmp.dir")
  *     .defaultValue("/tmp");
- * 
+ *
  * // simple integer-valued option with a default value
  * ConfigOption<Integer> parallelism = ConfigOptions
  *     .key("application.parallelism")
  *     .defaultValue(100);
- * 
+ *
  * // option with no default value
  * ConfigOption<String> userName = ConfigOptions
  *     .key("user.name")
  *     .noDefaultValue();
- * 
+ *
  * // option with deprecated keys to check
  * ConfigOption<Double> threshold = ConfigOptions
  *     .key("cpu.utilization.threshold")
@@ -54,7 +54,7 @@ public class ConfigOptions {
 
 	/**
 	 * Starts building a new {@link ConfigOption}.
-	 * 
+	 *
 	 * @param key The key for the config option.
 	 * @return The builder for the config option with the given key.
 	 */
@@ -71,7 +71,7 @@ public class ConfigOptions {
 	 */
 	public static final class OptionBuilder {
 
-		/** The key for the config option */
+		/** The key for the config option. */
 		private final String key;
 
 		/**
@@ -84,24 +84,24 @@ public class ConfigOptions {
 
 		/**
 		 * Creates a ConfigOption with the given default value.
-		 * 
+		 *
 		 * <p>This method does not accept "null". For options with no default value, choose
 		 * one of the {@code noDefaultValue} methods.
-		 * 
+		 *
 		 * @param value The default value for the config option
 		 * @param <T> The type of the default value.
 		 * @return The config option with the default value.
 		 */
 		public <T> ConfigOption<T> defaultValue(T value) {
 			checkNotNull(value);
-			return new ConfigOption<T>(key, value);
+			return new ConfigOption<>(key, value);
 		}
 
 		/**
 		 * Creates a string-valued option with no default value.
 		 * String-valued options are the only ones that can have no
 		 * default value.
-		 * 
+		 *
 		 * @return The created ConfigOption.
 		 */
 		public ConfigOption<String> noDefaultValue() {
@@ -111,6 +111,6 @@ public class ConfigOptions {
 
 	// ------------------------------------------------------------------------
 
-	/** Not intended to be instantiated */
+	/** Not intended to be instantiated. */
 	private ConfigOptions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptionsDocGenerator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptionsDocGenerator.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptionsDocGenerator.java
index 1769c35..02ce7bb 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptionsDocGenerator.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptionsDocGenerator.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
 import org.apache.flink.annotation.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
index dfcd04f..7d99fbb 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
@@ -18,14 +18,6 @@
 
 package org.apache.flink.configuration;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
 import org.apache.flink.annotation.Public;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -33,14 +25,23 @@ import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.StringValue;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
 /**
  * Lightweight configuration object which stores key/value pairs.
  */
 @Public
-public class Configuration extends ExecutionConfig.GlobalJobParameters 
+public class Configuration extends ExecutionConfig.GlobalJobParameters
 		implements IOReadableWritable, java.io.Serializable, Cloneable {
 
 	private static final long serialVersionUID = 1L;
@@ -66,29 +67,29 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	 * Creates a new empty configuration.
 	 */
 	public Configuration() {
-		this.confData = new HashMap<String, Object>();
+		this.confData = new HashMap<>();
 	}
 
 	/**
 	 * Creates a new configuration with the copy of the given configuration.
-	 * 
+	 *
 	 * @param other The configuration to copy the entries from.
 	 */
 	public Configuration(Configuration other) {
-		this.confData = new HashMap<String, Object>(other.confData);
+		this.confData = new HashMap<>(other.confData);
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 
 	/**
 	 * Returns the class associated with the given key as a string.
-	 * 
+	 *
 	 * @param <T> The type of the class to return.
 
 	 * @param key The key pointing to the associated value
 	 * @param defaultValue The optional default value returned if no entry exists
 	 * @param classLoader The class loader used to resolve the class.
-	 * 
+	 *
 	 * @return The value associated with the given key, or the default value, if to entry for the key exists.
 	 */
 	@SuppressWarnings("unchecked")
@@ -97,11 +98,11 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return (Class<T>) defaultValue;
 		}
-		
+
 		if (o.getClass() == String.class) {
 			return (Class<T>) Class.forName((String) o, true, classLoader);
 		}
-		
+
 		LOG.warn("Configuration cannot evaluate value " + o + " as a class name");
 		return (Class<T>) defaultValue;
 	}
@@ -109,7 +110,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	/**
 	 * Adds the given key/value pair to the configuration object. The class can be retrieved by invoking
 	 * {@link #getClass(String, Class, ClassLoader)} if it is in the scope of the class loader on the caller.
-	 * 
+	 *
 	 * @param key The key of the pair to be added
 	 * @param klazz The value of the pair to be added
 	 * @see #getClass(String, Class, ClassLoader)
@@ -120,7 +121,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a string.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -164,7 +165,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -190,7 +191,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as an integer.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -238,7 +239,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -264,7 +265,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a long.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -312,7 +313,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -338,7 +339,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a boolean.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -386,7 +387,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -412,7 +413,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a float.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -460,7 +461,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -486,7 +487,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a double.
-	 * 
+	 *
 	 * @param key
 	 *        the key pointing to the associated value
 	 * @param defaultValue
@@ -534,7 +535,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Adds the given key/value pair to the configuration object.
-	 * 
+	 *
 	 * @param key
 	 *        the key of the key/value pair to be added
 	 * @param value
@@ -560,7 +561,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 	/**
 	 * Returns the value associated with the given key as a byte array.
-	 * 
+	 *
 	 * @param key
 	 *        The key pointing to the associated value.
 	 * @param defaultValue
@@ -569,7 +570,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	 */
 	@SuppressWarnings("EqualsBetweenInconvertibleTypes")
 	public byte[] getBytes(String key, byte[] defaultValue) {
-		
+
 		Object o = getRawValue(key);
 		if (o == null) {
 			return defaultValue;
@@ -582,10 +583,10 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			return defaultValue;
 		}
 	}
-	
+
 	/**
 	 * Adds the given byte array to the configuration object. If key is <code>null</code> then nothing is added.
-	 * 
+	 *
 	 * @param key
 	 *        The key under which the bytes are added.
 	 * @param bytes
@@ -608,16 +609,16 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Returns the keys of all key/value pairs stored inside this
 	 * configuration object.
-	 * 
+	 *
 	 * @return the keys of all key/value pairs stored inside this configuration object
 	 */
 	public Set<String> keySet() {
 		synchronized (this.confData) {
-			return new HashSet<String>(this.confData.keySet());
+			return new HashSet<>(this.confData.keySet());
 		}
 	}
 
@@ -639,11 +640,11 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			}
 		}
 	}
-	
+
 	/**
 	 * Adds all entries from the given configuration into this configuration. The keys
 	 * are prepended with the given prefix.
-	 * 
+	 *
 	 * @param other
 	 *        The configuration whose entries are added to this configuration.
 	 * @param prefix
@@ -674,7 +675,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
-	 * Checks whether there is an entry with the specified key
+	 * Checks whether there is an entry with the specified key.
 	 *
 	 * @param key key of entry
 	 * @return true if the key is stored, false otherwise
@@ -686,7 +687,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
-	 * Checks whether there is an entry for the given config option
+	 * Checks whether there is an entry for the given config option.
 	 *
 	 * @param configOption The configuration option
 	 *
@@ -720,8 +721,8 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	@Override
 	public Map<String, String> toMap() {
 		synchronized (this.confData){
-			Map<String, String> ret = new HashMap<String, String>(this.confData.size());
-			for(Map.Entry<String, Object> entry : confData.entrySet()) {
+			Map<String, String> ret = new HashMap<>(this.confData.size());
+			for (Map.Entry<String, Object> entry : confData.entrySet()) {
 				ret.put(entry.getKey(), entry.getValue().toString());
 			}
 			return ret;
@@ -730,7 +731,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	<T> void setValueInternal(String key, T value) {
 		if (key == null) {
 			throw new NullPointerException("Key must not be null.");
@@ -738,17 +739,17 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (value == null) {
 			throw new NullPointerException("Value must not be null.");
 		}
-		
+
 		synchronized (this.confData) {
 			this.confData.put(key, value);
 		}
 	}
-	
+
 	private Object getRawValue(String key) {
 		if (key == null) {
 			throw new NullPointerException("Key must not be null.");
 		}
-		
+
 		synchronized (this.confData) {
 			return this.confData.get(key);
 		}
@@ -767,7 +768,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			for (String deprecatedKey : configOption.deprecatedKeys()) {
 				Object oo = getRawValue(deprecatedKey);
 				if (oo != null) {
-					LOG.warn("Config uses deprecated configuration key '{}' instead of proper key '{}'", 
+					LOG.warn("Config uses deprecated configuration key '{}' instead of proper key '{}'",
 							deprecatedKey, configOption.key());
 					return oo;
 				}
@@ -893,7 +894,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			for (int i = 0; i < numberOfProperties; i++) {
 				String key = StringValue.readString(in);
 				Object value;
-				
+
 				byte type = in.readByte();
 				switch (type) {
 					case TYPE_STRING:
@@ -922,7 +923,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 					default:
 						throw new IOException("Unrecognized type: " + type);
 				}
-				
+
 				this.confData.put(key, value);
 			}
 		}
@@ -932,14 +933,14 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	public void write(final DataOutputView out) throws IOException {
 		synchronized (this.confData) {
 			out.writeInt(this.confData.size());
-			
+
 			for (Map.Entry<String, Object> entry : this.confData.entrySet()) {
 				String key = entry.getKey();
 				Object val = entry.getValue();
-						
+
 				StringValue.writeString(key, out);
 				Class<?> clazz = val.getClass();
-				
+
 				if (clazz == String.class) {
 					out.write(TYPE_STRING);
 					StringValue.writeString((String) val, out);
@@ -976,7 +977,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			}
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 
 	@Override
@@ -996,11 +997,11 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		}
 		else if (obj instanceof Configuration) {
 			Map<String, Object> otherConf = ((Configuration) obj).confData;
-			
+
 			for (Map.Entry<String, Object> e : this.confData.entrySet()) {
 				Object thisVal = e.getValue();
 				Object otherVal = otherConf.get(e.getKey());
-				
+
 				if (!thisVal.getClass().equals(byte[].class)) {
 					if (!thisVal.equals(otherVal)) {
 						return false;
@@ -1013,14 +1014,14 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 					return false;
 				}
 			}
-			
+
 			return true;
 		}
 		else {
 			return false;
 		}
 	}
-	
+
 	@Override
 	public String toString() {
 		return this.confData.toString();

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
index f31ad8c..1182ed5 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java
@@ -81,7 +81,6 @@ public class CoreOptions {
 		.key("classloader.parent-first-patterns")
 		.defaultValue("java.;scala.;org.apache.flink.;javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback");
 
-
 	// ------------------------------------------------------------------------
 	//  process parameters
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
index 8cac66c..7b75c7a 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
@@ -111,7 +111,7 @@ public final class DelegatingConfiguration extends Configuration {
 	public int getInteger(ConfigOption<Integer> configOption) {
 		return  this.backingConfig.getInteger(prefixOption(configOption, prefix));
 	}
-	
+
 	@Override
 	public int getInteger(ConfigOption<Integer> configOption, int overrideDefault) {
 		return this.backingConfig.getInteger(configOption, overrideDefault);
@@ -282,7 +282,7 @@ public final class DelegatingConfiguration extends Configuration {
 			return this.backingConfig.keySet();
 		}
 
-		final HashSet<String> set = new HashSet<String>();
+		final HashSet<String> set = new HashSet<>();
 		int prefixLen = this.prefix.length();
 
 		for (String key : this.backingConfig.keySet()) {
@@ -307,7 +307,7 @@ public final class DelegatingConfiguration extends Configuration {
 			prefixed.put(prefix + entry.getKey(), entry.getValue());
 		}
 
-		return prefixed; 
+		return prefixed;
 	}
 
 	@Override
@@ -367,7 +367,7 @@ public final class DelegatingConfiguration extends Configuration {
 		}
 
 		String[] deprecated = deprecatedKeys.toArray(new String[deprecatedKeys.size()]);
-		return new ConfigOption<T>(key,
+		return new ConfigOption<>(key,
 			option.description(),
 			option.defaultValue(),
 			deprecated);

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
index 4569ebe..2f2a9cf 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
@@ -18,18 +18,19 @@
 
 package org.apache.flink.configuration;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-
 import org.apache.flink.annotation.Internal;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
 /**
  * Global configuration object for Flink. Similar to Java properties configuration
  * objects it includes key-value pairs which represent the framework's configuration.
@@ -65,8 +66,8 @@ public final class GlobalConfiguration {
 
 	/**
 	 * Loads the configuration files from the specified directory.
-	 * <p>
-	 * YAML files are supported as configuration files.
+	 *
+	 * <p>YAML files are supported as configuration files.
 	 *
 	 * @param configDir
 	 *        the directory which contains the configuration files
@@ -132,21 +133,21 @@ public final class GlobalConfiguration {
 
 	/**
 	 * Loads a YAML-file of key-value pairs.
-	 * <p>
-	 * Colon and whitespace ": " separate key and value (one per line). The hash tag "#" starts a single-line comment.
-	 * <p>
-	 * Example:
-	 * 
+	 *
+	 * <p>Colon and whitespace ": " separate key and value (one per line). The hash tag "#" starts a single-line comment.
+	 *
+	 * <p>Example:
+	 *
 	 * <pre>
 	 * jobmanager.rpc.address: localhost # network address for communication with the job manager
 	 * jobmanager.rpc.port   : 6123      # network port to connect to for communication with the job manager
 	 * taskmanager.rpc.port  : 6122      # network port the task manager expects incoming IPC connections
 	 * </pre>
-	 * <p>
-	 * This does not span the whole YAML specification, but only the *syntax* of simple YAML key-value pairs (see issue
+	 *
+	 * <p>This does not span the whole YAML specification, but only the *syntax* of simple YAML key-value pairs (see issue
 	 * #113 on GitHub). If at any point in time, there is a need to go beyond simple key-value pairs syntax
 	 * compatibility will allow to introduce a YAML parser library.
-	 * 
+	 *
 	 * @param file the YAML file to read from
 	 * @see <a href="http://www.yaml.org/spec/1.2/spec.html">YAML 1.2 specification</a>
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/HeartbeatManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HeartbeatManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HeartbeatManagerOptions.java
index 81cbc5d..e2f6ff6 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/HeartbeatManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HeartbeatManagerOptions.java
@@ -28,18 +28,18 @@ import static org.apache.flink.configuration.ConfigOptions.key;
 @PublicEvolving
 public class HeartbeatManagerOptions {
 
-	/** Time interval for requesting heartbeat from sender side */
+	/** Time interval for requesting heartbeat from sender side. */
 	public static final ConfigOption<Long> HEARTBEAT_INTERVAL =
 			key("heartbeat.interval")
 			.defaultValue(10000L);
 
-	/** Timeout for requesting and receiving heartbeat for both sender and receiver sides */
+	/** Timeout for requesting and receiving heartbeat for both sender and receiver sides. */
 	public static final ConfigOption<Long> HEARTBEAT_TIMEOUT =
 			key("heartbeat.timeout")
 			.defaultValue(50000L);
 
 	// ------------------------------------------------------------------------
 
-	/** Not intended to be instantiated */
+	/** Not intended to be instantiated. */
 	private HeartbeatManagerOptions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/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 6ee9f94..0039fdd 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
@@ -32,33 +32,33 @@ public class HighAvailabilityOptions {
 	//  Required High Availability Options
 	// ------------------------------------------------------------------------
 
-	/** 
+	/**
 	 * Defines high-availability mode used for the cluster execution.
 	 * A value of "NONE" signals no highly available setup.
 	 * To enable high-availability, set this mode to "ZOOKEEPER".
 	 */
-	public static final ConfigOption<String> HA_MODE = 
+	public static final ConfigOption<String> HA_MODE =
 			key("high-availability")
 			.defaultValue("NONE")
 			.withDeprecatedKeys("recovery.mode");
 
 	/**
-	 * The ID of the Flink cluster, used to separate multiple Flink clusters 
+	 * The ID of the Flink cluster, used to separate multiple Flink clusters
 	 * Needs to be set for standalone clusters, is automatically inferred in YARN and Mesos.
 	 */
-	public static final ConfigOption<String> HA_CLUSTER_ID = 
+	public static final ConfigOption<String> HA_CLUSTER_ID =
 			key("high-availability.cluster-id")
 			.defaultValue("/default")
 			.withDeprecatedKeys("high-availability.zookeeper.path.namespace", "recovery.zookeeper.path.namespace");
 
 	/**
-	 * File system path (URI) where Flink persists metadata in high-availability setups
+	 * File system path (URI) where Flink persists metadata in high-availability setups.
 	 */
 	public static final ConfigOption<String> HA_STORAGE_PATH =
 			key("high-availability.storageDir")
 			.noDefaultValue()
 			.withDeprecatedKeys("high-availability.zookeeper.storageDir", "recovery.zookeeper.storageDir");
-	
+
 
 	// ------------------------------------------------------------------------
 	//  Recovery Options
@@ -67,7 +67,7 @@ public class HighAvailabilityOptions {
 	/**
 	 * Optional port (range) used by the job manager in high-availability mode.
 	 */
-	public static final ConfigOption<String> HA_JOB_MANAGER_PORT_RANGE = 
+	public static final ConfigOption<String> HA_JOB_MANAGER_PORT_RANGE =
 			key("high-availability.jobmanager.port")
 			.defaultValue("0")
 			.withDeprecatedKeys("recovery.jobmanager.port");
@@ -75,7 +75,7 @@ public class HighAvailabilityOptions {
 	/**
 	 * The time before a JobManager after a fail over recovers the current jobs.
 	 */
-	public static final ConfigOption<String> HA_JOB_DELAY = 
+	public static final ConfigOption<String> HA_JOB_DELAY =
 			key("high-availability.job.delay")
 			.noDefaultValue()
 			.withDeprecatedKeys("recovery.job.delay");
@@ -93,7 +93,7 @@ public class HighAvailabilityOptions {
 			.withDeprecatedKeys("recovery.zookeeper.quorum");
 
 	/**
-	 * The root path under which Flink stores its entries in ZooKeeper
+	 * The root path under which Flink stores its entries in ZooKeeper.
 	 */
 	public static final ConfigOption<String> HA_ZOOKEEPER_ROOT =
 			key("high-availability.zookeeper.path.root")
@@ -139,7 +139,7 @@ public class HighAvailabilityOptions {
 	//  ZooKeeper Client Settings
 	// ------------------------------------------------------------------------
 
-	public static final ConfigOption<Integer> ZOOKEEPER_SESSION_TIMEOUT = 
+	public static final ConfigOption<Integer> ZOOKEEPER_SESSION_TIMEOUT =
 			key("high-availability.zookeeper.client.session-timeout")
 			.defaultValue(60000)
 			.withDeprecatedKeys("recovery.zookeeper.client.session-timeout");
@@ -149,17 +149,17 @@ public class HighAvailabilityOptions {
 			.defaultValue(15000)
 			.withDeprecatedKeys("recovery.zookeeper.client.connection-timeout");
 
-	public static final ConfigOption<Integer> ZOOKEEPER_RETRY_WAIT = 
+	public static final ConfigOption<Integer> ZOOKEEPER_RETRY_WAIT =
 			key("high-availability.zookeeper.client.retry-wait")
 			.defaultValue(5000)
 			.withDeprecatedKeys("recovery.zookeeper.client.retry-wait");
 
-	public static final ConfigOption<Integer> ZOOKEEPER_MAX_RETRY_ATTEMPTS = 
+	public static final ConfigOption<Integer> ZOOKEEPER_MAX_RETRY_ATTEMPTS =
 			key("high-availability.zookeeper.client.max-retry-attempts")
 			.defaultValue(3)
 			.withDeprecatedKeys("recovery.zookeeper.client.max-retry-attempts");
 
-	public static final ConfigOption<String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH = 
+	public static final ConfigOption<String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH =
 			key("high-availability.zookeeper.path.running-registry")
 			.defaultValue("/running_job_registry/");
 
@@ -169,6 +169,6 @@ public class HighAvailabilityOptions {
 
 	// ------------------------------------------------------------------------
 
-	/** Not intended to be instantiated */
+	/** Not intended to be instantiated. */
 	private HighAvailabilityOptions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java
index 27c56d4..c7c6933 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
 import org.apache.flink.annotation.PublicEvolving;

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/IllegalConfigurationException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/IllegalConfigurationException.java b/flink-core/src/main/java/org/apache/flink/configuration/IllegalConfigurationException.java
index fb1e5a8..6f85c16 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/IllegalConfigurationException.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/IllegalConfigurationException.java
@@ -33,7 +33,7 @@ public class IllegalConfigurationException extends RuntimeException {
 
 	/**
 	 * Constructs an new IllegalConfigurationException with the given error message.
-	 * 
+	 *
 	 * @param message The error message for the exception.
 	 */
 	public IllegalConfigurationException(String message) {

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
index ef3306e..1e22a24 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.configuration;
 
-import static org.apache.flink.configuration.ConfigOptions.key;
-
 import org.apache.flink.annotation.PublicEvolving;
 
+import static org.apache.flink.configuration.ConfigOptions.key;
+
 /**
  * Configuration options for the JobManager.
  */
@@ -31,8 +31,8 @@ public class JobManagerOptions {
 	/**
 	 * The config parameter defining the network address to connect to
 	 * for communication with the job manager.
-	 * 
-	 * <p>This value is only interpreted in setups where a single JobManager with static 
+	 *
+	 * <p>This value is only interpreted in setups where a single JobManager with static
 	 * name or address exists (simple standalone setups, or container setups with dynamic
 	 * service name resolution). It is not used in many high-availability setups, when a
 	 * leader-election service (like ZooKeeper) is used to elect and discover the JobManager
@@ -45,7 +45,7 @@ public class JobManagerOptions {
 	/**
 	 * The config parameter defining the network port to connect to
 	 * for communication with the job manager.
-	 * 
+	 *
 	 * <p>Like {@link JobManagerOptions#ADDRESS}, this value is only interpreted in setups where
 	 * a single JobManager with static name/address and port exists (simple standalone setups,
 	 * or container setups with dynamic service name resolution).
@@ -58,7 +58,7 @@ public class JobManagerOptions {
 		.defaultValue(6123);
 
 	/**
-	 * JVM heap size (in megabytes) for the JobManager
+	 * JVM heap size (in megabytes) for the JobManager.
 	 */
 	public static final ConfigOption<Integer> JOB_MANAGER_HEAP_MEMORY =
 		key("jobmanager.heap.mb")
@@ -83,7 +83,7 @@ public class JobManagerOptions {
 	 * This option specifies the interval in order to trigger a resource manager reconnection if the connection
 	 * to the resource manager has been lost.
 	 *
-	 * This option is only intended for internal use.
+	 * <p>This option is only intended for internal use.
 	 */
 	public static final ConfigOption<Long> RESOURCE_MANAGER_RECONNECT_INTERVAL =
 		key("jobmanager.resourcemanager.reconnect-interval")

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java
index ff38837..fba2a68 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java
@@ -27,14 +27,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * MemorySize is a representation of a number of bytes, viewable in different units.
- * 
+ *
  * <h2>Parsing</h2>
- * 
- * The size can be parsed from a text expression. If the expression is a pure number,
+ *
+ * <p>The size can be parsed from a text expression. If the expression is a pure number,
  * the value will be interpreted as bytes.
- * 
+ *
  * <p>To make larger values more compact, the common size suffixes are supported:
- * 
+ *
  * <ul>
  *     <li>q or 1b or 1bytes (bytes)
  *     <li>1k or 1kb or 1kibibytes (interpreted as kibibytes = 1024 bytes)
@@ -63,12 +63,12 @@ public class MemorySize implements java.io.Serializable {
 
 	// ------------------------------------------------------------------------
 
-	/** The memory size, in bytes */
+	/** The memory size, in bytes. */
 	private final long bytes;
 
 	/**
 	 * Constructs a new MemorySize.
-	 * 
+	 *
 	 * @param bytes The size, in bytes. Must be zero or larger.
 	 */
 	public MemorySize(long bytes) {
@@ -122,7 +122,7 @@ public class MemorySize implements java.io.Serializable {
 
 	@Override
 	public boolean equals(Object obj) {
-		return obj == this || 
+		return obj == this ||
 				(obj != null && obj.getClass() == this.getClass() && ((MemorySize) obj).bytes == this.bytes);
 	}
 
@@ -138,10 +138,10 @@ public class MemorySize implements java.io.Serializable {
 	/**
 	 * Parses the given string as as MemorySize.
 	 * The supported expressions are listed under {@link MemorySize}.
-	 * 
+	 *
 	 * @param text The string to parse
 	 * @return The parsed MemorySize
-	 * 
+	 *
 	 * @throws IllegalArgumentException Thrown, if the expression cannot be parsed.
 	 */
 	public static MemorySize parse(String text) throws IllegalArgumentException {
@@ -151,10 +151,10 @@ public class MemorySize implements java.io.Serializable {
 	/**
 	 * Parses the given string as bytes.
 	 * The supported expressions are listed under {@link MemorySize}.
-	 * 
+	 *
 	 * @param text The string to parse
 	 * @return The parsed size, in bytes.
-	 * 
+	 *
 	 * @throws IllegalArgumentException Thrown, if the expression cannot be parsed.
 	 */
 	public static long parseBytes(String text) throws IllegalArgumentException {
@@ -208,7 +208,7 @@ public class MemorySize implements java.io.Serializable {
 				multiplier = 1024L * 1024L * 1024L * 1024L;
 			}
 			else {
-				throw new IllegalArgumentException("Memory size unit '" + unit + 
+				throw new IllegalArgumentException("Memory size unit '" + unit +
 						"' does not match any of the recognized units: " + ALL_UNITS);
 			}
 		}
@@ -217,7 +217,7 @@ public class MemorySize implements java.io.Serializable {
 
 		// check for overflow
 		if (result / multiplier != value) {
-			throw new IllegalArgumentException("The value '" + text + 
+			throw new IllegalArgumentException("The value '" + text +
 					"' cannot be re represented as 64bit number of bytes (numeric overflow).");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
index 42eb575..24655fe 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
@@ -15,12 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
 import org.apache.flink.annotation.PublicEvolving;
 
 import static org.apache.flink.configuration.ConfigOptions.key;
 
+/**
+ * Configuration options for metrics and metric reporters.
+ */
 @PublicEvolving
 public class MetricOptions {
 
@@ -78,7 +82,7 @@ public class MetricOptions {
 		key("metrics.scope.operator")
 			.defaultValue("<host>.taskmanager.<tm_id>.<job_name>.<operator_name>.<subtask_index>");
 
-	/** The number of measured latencies to maintain at each operator */
+	/** The number of measured latencies to maintain at each operator. */
 	public static final ConfigOption<Integer> LATENCY_HISTORY_SIZE =
 		key("metrics.latency.history-size")
 			.defaultValue(128);

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
index 7b3b551..1c943b6 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
@@ -84,12 +84,12 @@ public class ResourceManagerOptions {
 
 	/**
 	 * Similar to the {@see CONTAINERIZED_MASTER_ENV_PREFIX}, this configuration prefix allows
-	 * setting custom environment variables for the workers (TaskManagers)
+	 * setting custom environment variables for the workers (TaskManagers).
 	 */
 	public static final String CONTAINERIZED_TASK_MANAGER_ENV_PREFIX = "containerized.taskmanager.env.";
-	
+
 	// ---------------------------------------------------------------------------------------------
 
-	/** Not intended to be instantiated */
+	/** Not intended to be instantiated. */
 	private ResourceManagerOptions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
index a2a2013..906e266 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
@@ -27,6 +27,7 @@ import static org.apache.flink.configuration.ConfigOptions.key;
  */
 @Internal
 public class RestOptions {
+
 	/**
 	 * The address that the server binds itself to / the client connects to.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
index fef0975..f2c2289 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
@@ -35,14 +35,14 @@ public class TaskManagerOptions {
 	// @TODO Migrate 'taskmanager.*' config options from ConfigConstants
 
 	/**
-	 * JVM heap size (in megabytes) for the TaskManagers
+	 * JVM heap size (in megabytes) for the TaskManagers.
 	 */
 	public static final ConfigOption<Integer> TASK_MANAGER_HEAP_MEMORY =
 			key("taskmanager.heap.mb")
 			.defaultValue(1024);
 
 	/**
-	 * Whether to kill the TaskManager when the task thread throws an OutOfMemoryError
+	 * Whether to kill the TaskManager when the task thread throws an OutOfMemoryError.
 	 */
 	public static final ConfigOption<Boolean> KILL_ON_OUT_OF_MEMORY =
 			key("taskmanager.jvm-exit-on-oom")
@@ -61,7 +61,7 @@ public class TaskManagerOptions {
 	 * The default network port range the task manager expects incoming IPC connections. The {@code "0"} means that
 	 * the TaskManager searches for a free port.
 	 */
-	public static final ConfigOption<String> RPC_PORT = 
+	public static final ConfigOption<String> RPC_PORT =
 		key("taskmanager.rpc.port")
 			.defaultValue("0");
 
@@ -131,14 +131,14 @@ public class TaskManagerOptions {
 			.defaultValue(0.1f);
 
 	/**
-	 * Minimum memory size for network buffers (in bytes)
+	 * Minimum memory size for network buffers (in bytes).
 	 */
 	public static final ConfigOption<Long> NETWORK_BUFFERS_MEMORY_MIN =
 			key("taskmanager.network.memory.min")
 			.defaultValue(64L << 20); // 64 MB
 
 	/**
-	 * Maximum memory size for network buffers (in bytes)
+	 * Maximum memory size for network buffers (in bytes).
 	 */
 	public static final ConfigOption<Long> NETWORK_BUFFERS_MEMORY_MAX =
 			key("taskmanager.network.memory.max")
@@ -147,7 +147,7 @@ public class TaskManagerOptions {
 	/**
 	 * Number of network buffers to use for each outgoing/incoming channel (subpartition/input channel).
 	 *
-	 * Reasoning: 1 buffer for in-flight data in the subpartition + 1 buffer for parallel serialization
+	 * <p>Reasoning: 1 buffer for in-flight data in the subpartition + 1 buffer for parallel serialization.
 	 */
 	public static final ConfigOption<Integer> NETWORK_BUFFERS_PER_CHANNEL =
 			key("taskmanager.network.memory.buffers-per-channel")
@@ -219,6 +219,6 @@ public class TaskManagerOptions {
 
 	// ------------------------------------------------------------------------
 
-	/** Not intended to be instantiated */
+	/** Not intended to be instantiated. */
 	private TaskManagerOptions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java
index 24ad61e..f92de1c 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java
@@ -27,13 +27,13 @@ import java.util.Properties;
  */
 @Public
 public class UnmodifiableConfiguration extends Configuration {
-	
+
 	private static final long serialVersionUID = -8151292629158972280L;
 
 	/**
 	 * Creates a new UnmodifiableConfiguration, which holds a copy of the given configuration
 	 * that cannot be altered.
-	 * 
+	 *
 	 * @param config The configuration with the original contents.
 	 */
 	public UnmodifiableConfiguration(Configuration config) {
@@ -44,7 +44,6 @@ public class UnmodifiableConfiguration extends Configuration {
 	//  All mutating methods must fail
 	// --------------------------------------------------------------------------------------------
 
-
 	@Override
 	public void addAllToProperties(Properties props) {
 		// override to make the UnmodifiableConfigurationTest happy

http://git-wip-us.apache.org/repos/asf/flink/blob/1f9c2d97/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java
index 3733244..b74f23f 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java
@@ -27,6 +27,7 @@ import static org.apache.flink.configuration.ConfigOptions.key;
  */
 @PublicEvolving
 public class WebOptions {
+
 	/**
 	 * Config parameter defining the runtime monitor web-frontend server address.
 	 */
@@ -61,7 +62,7 @@ public class WebOptions {
 			.withDeprecatedKeys("jobmanager.web.refresh-interval");
 
 	/**
-	 * Config parameter to override SSL support for the JobManager Web UI
+	 * Config parameter to override SSL support for the JobManager Web UI.
 	 */
 	public static final ConfigOption<Boolean> SSL_ENABLED =
 		key("web.ssl.enabled")
@@ -156,8 +157,8 @@ public class WebOptions {
 		.key("web.timeout")
 		.defaultValue(10L * 1000L);
 
+	// ------------------------------------------------------------------------
 
-	private WebOptions() {
-		throw new IllegalAccessError();
-	}
+	/** Not meant to be instantiated. */
+	private WebOptions() {}
 }


[16/17] flink git commit: [hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.util'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
index 6638062..208a301 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.util;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
-
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -30,6 +26,10 @@ import org.apache.flink.types.StringValue;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -43,7 +43,7 @@ public final class StringUtils {
 	/**
 	 * Given an array of bytes it will convert the bytes to a hex string
 	 * representation of the bytes.
-	 * 
+	 *
 	 * @param bytes
 	 *        the bytes to convert in a hex string
 	 * @param start
@@ -56,8 +56,6 @@ public final class StringUtils {
 		if (bytes == null) {
 			throw new IllegalArgumentException("bytes == null");
 		}
-		
-		
 
 		int length = end - start;
 		char[] out = new char[length * 2];
@@ -73,7 +71,7 @@ public final class StringUtils {
 	/**
 	 * Given an array of bytes it will convert the bytes to a hex string
 	 * representation of the bytes.
-	 * 
+	 *
 	 * @param bytes
 	 *        the bytes to convert in a hex string
 	 * @return hex string representation of the byte array
@@ -85,7 +83,7 @@ public final class StringUtils {
 	/**
 	 * Given a hex string this will return the byte array corresponding to the
 	 * string .
-	 * 
+	 *
 	 * @param hex
 	 *        the hex String array
 	 * @return a byte array that is a hex string representation of the given
@@ -104,7 +102,7 @@ public final class StringUtils {
 	 * object is an array. In that case, it will use the {@link #arrayToString(Object)}
 	 * method to create a string representation of the array that includes all contained
 	 * elements.
-	 * 
+	 *
 	 * @param o The object for which to create the string representation.
 	 * @return The string representation of the object.
 	 */
@@ -115,14 +113,14 @@ public final class StringUtils {
 		if (o.getClass().isArray()) {
 			return arrayToString(o);
 		}
-		
+
 		return o.toString();
 	}
-	
+
 	/**
 	 * Returns a string representation of the given array. This method takes an Object
 	 * to allow also all types of primitive type arrays.
-	 * 
+	 *
 	 * @param array The array to create a string representation for.
 	 * @return The string representation of the array.
 	 * @throws IllegalArgumentException If the given object is no array.
@@ -131,7 +129,7 @@ public final class StringUtils {
 		if (array == null) {
 			throw new NullPointerException();
 		}
-		
+
 		if (array instanceof int[]) {
 			return Arrays.toString((int[]) array);
 		}
@@ -159,20 +157,20 @@ public final class StringUtils {
 		if (array instanceof short[]) {
 			return Arrays.toString((short[]) array);
 		}
-		
+
 		if (array.getClass().isArray()) {
 			return "<unknown array type>";
 		} else {
 			throw new IllegalArgumentException("The given argument is no array.");
 		}
 	}
-	
+
 	/**
 	 * Replaces control characters by their escape-coded version. For example,
 	 * if the string contains a line break character ('\n'), this character will
 	 * be replaced by the two characters backslash '\' and 'n'. As a consequence, the
 	 * resulting string will not contain any more control characters.
-	 * 
+	 *
 	 * @param str The string in which to replace the control characters.
 	 * @return The string with the replaced characters.
 	 */
@@ -205,11 +203,11 @@ public final class StringUtils {
 
 		return sb.toString();
 	}
-	
+
 	/**
 	 * Creates a random string with a length within the given interval. The string contains only characters that
 	 * can be represented as a single code point.
-	 * 
+	 *
 	 * @param rnd The random used to create the strings.
 	 * @param minLength The minimum string length.
 	 * @param maxLength The maximum string length (inclusive).
@@ -217,18 +215,18 @@ public final class StringUtils {
 	 */
 	public static String getRandomString(Random rnd, int minLength, int maxLength) {
 		int len = rnd.nextInt(maxLength - minLength + 1) + minLength;
-		
+
 		char[] data = new char[len];
 		for (int i = 0; i < data.length; i++) {
 			data[i] = (char) (rnd.nextInt(0x7fff) + 1);
 		}
 		return new String(data);
 	}
-	
+
 	/**
 	 * Creates a random string with a length within the given interval. The string contains only characters that
 	 * can be represented as a single code point.
-	 * 
+	 *
 	 * @param rnd The random used to create the strings.
 	 * @param minLength The minimum string length.
 	 * @param maxLength The maximum string length (inclusive).
@@ -238,10 +236,10 @@ public final class StringUtils {
 	 */
 	public static String getRandomString(Random rnd, int minLength, int maxLength, char minValue, char maxValue) {
 		int len = rnd.nextInt(maxLength - minLength + 1) + minLength;
-		
+
 		char[] data = new char[len];
 		int diff = maxValue - minValue + 1;
-		
+
 		for (int i = 0; i < data.length; i++) {
 			data[i] = (char) (rnd.nextInt(diff) + minValue);
 		}
@@ -254,7 +252,7 @@ public final class StringUtils {
 	 *
 	 * @param str The string to write
 	 * @param out The output to write to
-	 *   
+	 *
 	 * @throws IOException Thrown, if the writing or the serialization fails.
 	 */
 	public static void writeString(@Nonnull String str, DataOutputView out) throws IOException {
@@ -267,7 +265,7 @@ public final class StringUtils {
 	 *
 	 * @param in The input to read from
 	 * @return The deserialized String
-	 * 
+	 *
 	 * @throws IOException Thrown, if the reading or the deserialization fails.
 	 */
 	public static String readString(DataInputView in) throws IOException {
@@ -277,10 +275,10 @@ public final class StringUtils {
 	/**
 	 * Writes a String to the given output. The string may be null.
 	 * The written string can be read with {@link #readNullableString(DataInputView)}-
-	 * 
+	 *
 	 * @param str The string to write, or null.
 	 * @param out The output to write to.
-	 * 
+	 *
 	 * @throws IOException Thrown, if the writing or the serialization fails.
 	 */
 	public static void writeNullableString(@Nullable String str, DataOutputView out) throws IOException {
@@ -291,14 +289,14 @@ public final class StringUtils {
 			out.writeBoolean(false);
 		}
 	}
-	
+
 	/**
 	 * Reads a String from the given input. The string may be null and must have been written with
 	 * {@link #writeNullableString(String, DataOutputView)}.
-	 * 
+	 *
 	 * @param in The input to read from.
 	 * @return The deserialized string, or null.
-	 * 
+	 *
 	 * @throws IOException Thrown, if the reading or the deserialization fails.
 	 */
 	public static @Nullable String readNullableString(DataInputView in) throws IOException {
@@ -312,7 +310,7 @@ public final class StringUtils {
 	/**
 	 * Checks if the string is null, empty, or contains only whitespace characters.
 	 * A whitespace character is defined via {@link Character#isWhitespace(char)}.
-	 * 
+	 *
 	 * @param str The string to check
 	 * @return True, if the string is null or blank, false otherwise.
 	 */
@@ -334,11 +332,11 @@ public final class StringUtils {
 	 * If both string arguments are non-null, this method concatenates them with ' and '.
 	 * If only one of the arguments is non-null, this method returns the non-null argument.
 	 * If both arguments are null, this method returns null.
-	 * 
+	 *
 	 * @param s1 The first string argument
 	 * @param s2 The second string argument
-	 * 
-	 * @return The concatenated string, or non-null argument, or null 
+	 *
+	 * @return The concatenated string, or non-null argument, or null
 	 */
 	@Nullable
 	public static String concatenateWithAnd(@Nullable String s1, @Nullable String s2) {
@@ -346,12 +344,12 @@ public final class StringUtils {
 			return s2 == null ? s1 : s1 + " and " + s2;
 		}
 		else {
-			return s2 != null ? s2 : null;
+			return s2;
 		}
 	}
 
 	// ------------------------------------------------------------------------
 
-	/** Prevent instantiation of this utility class */
+	/** Prevent instantiation of this utility class. */
 	private StringUtils() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/StringValueUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringValueUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringValueUtils.java
index 03d955f..a44e7a2 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringValueUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringValueUtils.java
@@ -18,14 +18,14 @@
 
 package org.apache.flink.util;
 
-import java.io.Serializable;
-
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.types.StringValue;
 
+import java.io.Serializable;
+
 /**
  * Utility class for efficient operations on {@link StringValue}.
- * 
+ *
  * <p>All methods in this class are written to be optimized for efficiency and work directly
  * on the StringValues char arrays, avoiding copies. For simplicity and efficiency, the methods
  * only apply to strings whose characters are representable in a single <tt>char</tt>,
@@ -33,36 +33,36 @@ import org.apache.flink.types.StringValue;
  */
 @PublicEvolving
 public final class StringValueUtils {
-	
+
 	/**
 	 * Converts the given <code>StringValue</code> into a lower case variant.
-	 * 
+	 *
 	 * @param string The string to convert to lower case.
 	 */
 	public static void toLowerCase(StringValue string) {
 		final char[] chars = string.getCharArray();
 		final int len = string.length();
-		
+
 		for (int i = 0; i < len; i++) {
 			chars[i] = Character.toLowerCase(chars[i]);
 		}
 	}
-	
+
 	/**
 	 * Replaces all non-word characters in a string by a given character. The only
 	 * characters not replaced are the characters that qualify as word characters
 	 * or digit characters with respect to {@link Character#isLetter(char)} or
 	 * {@link Character#isDigit(char)}, as well as the underscore character.
-	 * <p>
-	 * This operation is intended to simplify strings for counting distinct words.
-	 * 
+	 *
+	 * <p>This operation is intended to simplify strings for counting distinct words.
+	 *
 	 * @param string The string value to have the non-word characters replaced.
 	 * @param replacement The character to use as the replacement.
 	 */
 	public static void replaceNonWordChars(StringValue string, char replacement) {
 		final char[] chars = string.getCharArray();
 		final int len = string.length();
-		
+
 		for (int i = 0; i < len; i++) {
 			final char c = chars[i];
 			if (!(Character.isLetter(c) || Character.isDigit(c) || c == '_')) {
@@ -70,9 +70,9 @@ public final class StringValueUtils {
 			}
 		}
 	}
-	
+
 	// ============================================================================================
-	
+
 	/**
 	 * A tokenizer for string values that uses whitespace characters as token delimiters.
 	 * The tokenizer is designed to have a resettable state and operate on mutable objects,
@@ -80,19 +80,19 @@ public final class StringValueUtils {
 	 */
 	public static final class WhitespaceTokenizer implements Serializable {
 		private static final long serialVersionUID = 1L;
-		
+
 		private StringValue toTokenize;		// the string to tokenize
 		private int pos;					// the current position in the string
 		private int limit;					// the limit in the string's character data
-		
+
 		/**
 		 * Creates a new tokenizer with an undefined internal state.
 		 */
 		public WhitespaceTokenizer() {}
-		
+
 		/**
 		 * Sets the string to be tokenized and resets the state of the tokenizer.
-		 * 
+		 *
 		 * @param string The string value to be tokenized.
 		 */
 		public void setStringToTokenize(StringValue string) {
@@ -100,11 +100,11 @@ public final class StringValueUtils {
 			this.pos = 0;
 			this.limit = string.length();
 		}
-		
+
 		/**
 		 * Gets the next token from the string. If another token is available, the token is stored
 		 * in the given target StringValue object.
-		 * 
+		 *
 		 * @param target The StringValue object to store the next token in.
 		 * @return True, if there was another token, false if not.
 		 */
@@ -112,28 +112,28 @@ public final class StringValueUtils {
 			final char[] data = this.toTokenize.getCharArray();
 			final int limit = this.limit;
 			int pos = this.pos;
-			
+
 			// skip the delimiter
 			for (; pos < limit && Character.isWhitespace(data[pos]); pos++) {
 			}
-			
+
 			if (pos >= limit) {
 				this.pos = pos;
 				return false;
 			}
-			
+
 			final int start = pos;
 			for (; pos < limit && !Character.isWhitespace(data[pos]); pos++) {
 			}
-			
+
 			this.pos = pos;
 			target.setValue(this.toTokenize, start, pos - start);
 			return true;
 		}
 	}
-	
+
 	// ============================================================================================
-	
+
 	/**
 	 * Private constructor to prevent instantiation, as this is a utility method encapsulating class.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/UnionIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/UnionIterator.java b/flink-core/src/main/java/org/apache/flink/util/UnionIterator.java
index 17204ce..afb32bc 100644
--- a/flink-core/src/main/java/org/apache/flink/util/UnionIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/UnionIterator.java
@@ -18,30 +18,39 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Internal;
+
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 
+/**
+ * An iterator that concatenates a collection of iterators.
+ * The UnionIterator is a mutable, reusable type.
+ *
+ * @param <T> The type returned by the iterator.
+ */
+@Internal
 public class UnionIterator<T> implements Iterator<T>, Iterable<T> {
-	
+
 	private Iterator<T> currentIterator;
-	
+
 	private ArrayList<Iterator<T>> furtherIterators = new ArrayList<>();
-	
+
 	private int nextIterator;
-	
+
 	private boolean iteratorAvailable = true;
 
 	// ------------------------------------------------------------------------
-	
+
 	public void clear() {
 		currentIterator = null;
 		furtherIterators.clear();
 		nextIterator = 0;
 		iteratorAvailable = true;
 	}
-	
+
 	public void addList(List<T> list) {
 		add(list.iterator());
 	}
@@ -54,9 +63,9 @@ public class UnionIterator<T> implements Iterator<T>, Iterable<T> {
 			furtherIterators.add(iterator);
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
-	
+
 	@Override
 	public Iterator<T> iterator() {
 		if (iteratorAvailable) {
@@ -81,7 +90,7 @@ public class UnionIterator<T> implements Iterator<T>, Iterable<T> {
 				currentIterator = null;
 			}
 		}
-		
+
 		return false;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/Visitable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Visitable.java b/flink-core/src/main/java/org/apache/flink/util/Visitable.java
index 4df6b6e..3f45376 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Visitable.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Visitable.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
@@ -24,8 +23,8 @@ import org.apache.flink.annotation.Internal;
 /**
  * This interface marks types as visitable during a traversal. The central method <i>accept(...)</i> contains the logic
  * about how to invoke the supplied {@link Visitor} on the visitable object, and how to traverse further.
- * <p>
- * This concept makes it easy to implement for example a depth-first traversal of a tree or DAG with different types of
+ *
+ * <p>This concept makes it easy to implement for example a depth-first traversal of a tree or DAG with different types of
  * logic during the traversal. The <i>accept(...)</i> method calls the visitor and then send the visitor to its children
  * (or predecessors). Using different types of visitors, different operations can be performed during the traversal, while
  * writing the actual traversal code only once.
@@ -34,16 +33,16 @@ import org.apache.flink.annotation.Internal;
  */
 @Internal
 public interface Visitable<T extends Visitable<T>> {
-	
+
 	/**
 	 * Contains the logic to invoke the visitor and continue the traversal.
 	 * Typically invokes the pre-visit method of the visitor, then sends the visitor to the children (or predecessors)
 	 * and then invokes the post-visit method.
-	 * <p>
-	 * A typical code example is the following:
+	 *
+	 * <p>A typical code example is the following:
 	 * <pre>{@code
 	 * public void accept(Visitor<Operator> visitor) {
-	 *     boolean descend = visitor.preVisit(this);	
+	 *     boolean descend = visitor.preVisit(this);
 	 *     if (descend) {
 	 *         if (this.input != null) {
 	 *             this.input.accept(visitor);
@@ -52,9 +51,9 @@ public interface Visitable<T extends Visitable<T>> {
 	 *     }
 	 * }
 	 * }</pre>
-	 * 
+	 *
 	 * @param visitor The visitor to be called with this object as the parameter.
-	 * 
+	 *
 	 * @see Visitor#preVisit(Visitable)
 	 * @see Visitor#postVisit(Visitable)
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/Visitor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Visitor.java b/flink-core/src/main/java/org/apache/flink/util/Visitor.java
index 2cc68fc..8d837d1 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Visitor.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Visitor.java
@@ -16,27 +16,25 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
 
 /**
- * A visitor encapsulates functionality that is applied to each node in the process of a traversal of a tree or DAD. 
+ * A visitor encapsulates functionality that is applied to each node in the process of a traversal of a tree or DAG.
  */
 @Internal
 public interface Visitor<T extends Visitable<T>> {
-	
+
 	/**
-	 * 
-	 * @param visitable
-	 * 
+	 * Method that is invoked on the visit before visiting and child nodes or descendant nodes.
+	 *
 	 * @return True, if the traversal should continue, false otherwise.
 	 */
 	boolean preVisit(T visitable);
 
 	/**
-	 * @param visitable
+	 * Method that is invoked after all child nodes or descendant nodes were visited.
 	 */
 	void postVisit(T visitable);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java b/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
index 213feee..6dd726f 100644
--- a/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
+++ b/flink-core/src/main/java/org/apache/flink/util/WrappingProxy.java
@@ -20,9 +20,14 @@ package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
 
+/**
+ * Interface for objects that wrap another object and proxy (possibly a subset) of
+ * the methods of that object.
+ *
+ * @param <T> The type that is wrapped.
+ */
 @Internal
 public interface WrappingProxy<T> {
 
 	T getWrappedDelegate();
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java b/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
index 7493d76..3fbd6df 100644
--- a/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/WrappingProxyUtil.java
@@ -20,6 +20,9 @@ package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
 
+/**
+ * Utilits for working with {@link WrappingProxy}.
+ */
 @Internal
 public final class WrappingProxyUtil {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java b/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java
index f9306df..7b230be 100644
--- a/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java
@@ -24,7 +24,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * A runtime exception that is explicitly used to wrap non-runtime exceptions.
- * 
+ *
  * <p>The exception is recognized (for example by the Task when reporting exceptions as
  * failure causes) and unwrapped to avoid including the wrapper's stack trace in the reports.
  * That way, exception traces are keeping to the important parts.
@@ -44,7 +44,7 @@ public class WrappingRuntimeException extends FlinkRuntimeException {
 	/**
 	 * Recursively unwraps this WrappingRuntimeException and its causes, getting the first
 	 * non wrapping exception.
-	 * 
+	 *
 	 * @return The first cause that is not a wrapping exception.
 	 */
 	public Throwable unwrap() {

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
index fbb2c24..20a46ad 100644
--- a/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/ExceptionUtilsTest.java
@@ -20,7 +20,11 @@ package org.apache.flink.util;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for the utility methods in {@link ExceptionUtils}.

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
index ada442f..67f831a 100644
--- a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
@@ -51,7 +51,7 @@ public class FileUtilsTest {
 	@Test
 	public void testDeletePathIfEmpty() throws IOException {
 		final FileSystem localFs = FileSystem.getLocalFileSystem();
-		
+
 		final File dir = tmp.newFolder();
 		assertTrue(dir.exists());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java
index cf27ca2..0e86391 100644
--- a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java
@@ -26,6 +26,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.DoubleValue;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.types.Value;
+
 import org.junit.Test;
 
 import java.io.IOException;
@@ -40,6 +41,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Tests for the {@link InstantiationUtil}.
+ */
 public class InstantiationUtilTest extends TestLogger {
 
 	@Test
@@ -88,14 +92,14 @@ public class InstantiationUtilTest extends TestLogger {
 
 		assertEquals("Serialized record is not equal after serialization.", toSerialize, deserialized);
 	}
-	
+
 	@Test
 	public void testWriteToConfigFailingSerialization() {
 		try {
 			final String key1 = "testkey1";
 			final String key2 = "testkey2";
 			final Configuration config = new Configuration();
-			
+
 			try {
 				InstantiationUtil.writeObjectToConfig(new TestClassWriteFails(), config, "irgnored");
 				fail("should throw an exception");
@@ -106,10 +110,10 @@ public class InstantiationUtilTest extends TestLogger {
 			catch (Exception e) {
 				fail("Wrong exception type - exception not properly forwarded");
 			}
-			
+
 			InstantiationUtil.writeObjectToConfig(new TestClassReadFails(), config, key1);
 			InstantiationUtil.writeObjectToConfig(new TestClassReadFailsCNF(), config, key2);
-			
+
 			try {
 				InstantiationUtil.readObjectFromConfig(config, key1, getClass().getClassLoader());
 				fail("should throw an exception");
@@ -120,7 +124,7 @@ public class InstantiationUtilTest extends TestLogger {
 			catch (Exception e) {
 				fail("Wrong exception type - exception not properly forwarded");
 			}
-			
+
 			try {
 				InstantiationUtil.readObjectFromConfig(config, key2, getClass().getClassLoader());
 				fail("should throw an exception");
@@ -142,48 +146,49 @@ public class InstantiationUtilTest extends TestLogger {
 	public void testCopyWritable() throws Exception {
 		WritableType original = new WritableType();
 		WritableType copy = InstantiationUtil.createCopyWritable(original);
-		
+
 		assertTrue(original != copy);
 		assertTrue(original.equals(copy));
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private class TestClass {}
-	
+
 	private static class TestException extends IOException {
 		private static final long serialVersionUID = 1L;
 	}
-	
+
 	private static class TestClassWriteFails implements java.io.Serializable {
-		
+
 		private static final long serialVersionUID = 1L;
 
 		private void writeObject(ObjectOutputStream out) throws IOException {
 			throw new TestException();
 		}
 	}
-	
+
 	private static class TestClassReadFails implements java.io.Serializable {
-		
+
 		private static final long serialVersionUID = 1L;
 
 		private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
 			throw new TestException();
 		}
 	}
-	
+
 	private static class TestClassReadFailsCNF implements java.io.Serializable {
-		
+
 		private static final long serialVersionUID = 1L;
 
 		private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
 			throw new ClassNotFoundException("test exception");
 		}
 	}
-	
+
+	/** A simple test type. */
 	public static final class WritableType implements IOReadableWritable {
-		
+
 		private int aInt;
 		private long aLong;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/LongValueSequenceIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/LongValueSequenceIteratorTest.java b/flink-core/src/test/java/org/apache/flink/util/LongValueSequenceIteratorTest.java
index 3407690..f738f5b 100644
--- a/flink-core/src/test/java/org/apache/flink/util/LongValueSequenceIteratorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/LongValueSequenceIteratorTest.java
@@ -18,12 +18,14 @@
 
 package org.apache.flink.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+/**
+ * Tests for the {@link LongValueSequenceIterator}.
+ */
 public class LongValueSequenceIteratorTest extends TestLogger {
 
 	@Test
@@ -49,25 +51,24 @@ public class LongValueSequenceIteratorTest extends TestLogger {
 		testSplitting(new org.apache.flink.util.LongValueSequenceIterator(10, 15), 10);
 	}
 
-	private static final void testSplitting(org.apache.flink.util.LongValueSequenceIterator iter, int numSplits) {
+	private static void testSplitting(org.apache.flink.util.LongValueSequenceIterator iter, int numSplits) {
 		org.apache.flink.util.LongValueSequenceIterator[] splits = iter.split(numSplits);
 
 		assertEquals(numSplits, splits.length);
 
 		// test start and end of range
 		assertEquals(iter.getCurrent(), splits[0].getCurrent());
-		assertEquals(iter.getTo(), splits[numSplits-1].getTo());
+		assertEquals(iter.getTo(), splits[numSplits - 1].getTo());
 
 		// test continuous range
 		for (int i = 1; i < splits.length; i++) {
-			assertEquals(splits[i-1].getTo() + 1, splits[i].getCurrent());
+			assertEquals(splits[i - 1].getTo() + 1, splits[i].getCurrent());
 		}
 
 		testMaxSplitDiff(splits);
 	}
 
-
-	private static final void testMaxSplitDiff(org.apache.flink.util.LongValueSequenceIterator[] iters) {
+	private static void testMaxSplitDiff(org.apache.flink.util.LongValueSequenceIterator[] iters) {
 		long minSplitSize = Long.MAX_VALUE;
 		long maxSplitSize = Long.MIN_VALUE;
 
@@ -86,7 +87,7 @@ public class LongValueSequenceIteratorTest extends TestLogger {
 			maxSplitSize = Math.max(maxSplitSize, diff);
 		}
 
-		assertTrue(maxSplitSize == minSplitSize || maxSplitSize-1 == minSplitSize);
+		assertTrue(maxSplitSize == minSplitSize || maxSplitSize - 1 == minSplitSize);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/MathUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/MathUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/MathUtilTest.java
index c98b7fc..82c17e9 100644
--- a/flink-core/src/test/java/org/apache/flink/util/MathUtilTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/MathUtilTest.java
@@ -18,14 +18,16 @@
 
 package org.apache.flink.util;
 
+import org.junit.Test;
+
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.flink.util.MathUtils;
-import org.junit.Test;
-
+/**
+ * Tests for the {@link MathUtils}.
+ */
 public class MathUtilTest {
 
 	@Test
@@ -43,14 +45,13 @@ public class MathUtilTest {
 		assertEquals(13, MathUtils.log2floor((0x1 << 13) + 1));
 		assertEquals(30, MathUtils.log2floor(Integer.MAX_VALUE));
 		assertEquals(31, MathUtils.log2floor(-1));
-		
+
 		try {
 			MathUtils.log2floor(0);
 			fail();
-		}
-		catch (ArithmeticException aex) {}
+		} catch (ArithmeticException ignored) {}
 	}
-	
+
 	@Test
 	public void testRoundDownToPowerOf2() {
 		assertEquals(0, MathUtils.roundDownToPowerOf2(0));

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java
index 03b21dd..04faa67 100644
--- a/flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java
@@ -30,8 +30,12 @@ import java.util.Set;
 
 import static org.hamcrest.core.IsCollectionContaining.hasItems;
 import static org.hamcrest.core.IsNot.not;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
+/**
+ * Tests for the {@link NetUtils}.
+ */
 public class NetUtilsTest {
 
 	@Test
@@ -62,16 +66,16 @@ public class NetUtilsTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testIPv4URLEncoding() {
 		try {
 			final String addressString = "10.244.243.12";
 			final int port = 23453;
-			
+
 			InetAddress address = InetAddress.getByName(addressString);
 			InetSocketAddress socketAddress = new InetSocketAddress(address, port);
-			
+
 			assertEquals(addressString, NetUtils.ipAddressToUrlString(address));
 			assertEquals(addressString + ':' + port, NetUtils.ipAddressAndPortToUrlString(address, port));
 			assertEquals(addressString + ':' + port, NetUtils.socketAddressToUrlString(socketAddress));
@@ -102,19 +106,17 @@ public class NetUtilsTest {
 		}
 	}
 
-
-
 	@Test
 	public void testFreePortRangeUtility() {
 		// inspired by Hadoop's example for "yarn.app.mapreduce.am.job.client.port-range"
 		String rangeDefinition = "50000-50050, 50100-50200,51234 "; // this also contains some whitespaces
 		Iterator<Integer> portsIter = NetUtils.getPortRangeFromString(rangeDefinition);
 		Set<Integer> ports = new HashSet<>();
-		while(portsIter.hasNext()) {
+		while (portsIter.hasNext()) {
 			Assert.assertTrue("Duplicate element", ports.add(portsIter.next()));
 		}
 
-		Assert.assertEquals(51+101+1, ports.size());
+		Assert.assertEquals(51 + 101 + 1, ports.size());
 		// check first range
 		Assert.assertThat(ports, hasItems(50000, 50001, 50002, 50050));
 		// check second range and last point
@@ -122,46 +124,58 @@ public class NetUtilsTest {
 		// check that only ranges are included
 		Assert.assertThat(ports, not(hasItems(50051, 50052, 1337, 50201, 49999, 50099)));
 
-
 		// test single port "range":
 		portsIter = NetUtils.getPortRangeFromString(" 51234");
 		Assert.assertTrue(portsIter.hasNext());
-		Assert.assertEquals(51234, (int)portsIter.next());
+		Assert.assertEquals(51234, (int) portsIter.next());
 		Assert.assertFalse(portsIter.hasNext());
 
 		// test port list
 		portsIter = NetUtils.getPortRangeFromString("5,1,2,3,4");
 		Assert.assertTrue(portsIter.hasNext());
-		Assert.assertEquals(5, (int)portsIter.next());
-		Assert.assertEquals(1, (int)portsIter.next());
-		Assert.assertEquals(2, (int)portsIter.next());
-		Assert.assertEquals(3, (int)portsIter.next());
-		Assert.assertEquals(4, (int)portsIter.next());
+		Assert.assertEquals(5, (int) portsIter.next());
+		Assert.assertEquals(1, (int) portsIter.next());
+		Assert.assertEquals(2, (int) portsIter.next());
+		Assert.assertEquals(3, (int) portsIter.next());
+		Assert.assertEquals(4, (int) portsIter.next());
 		Assert.assertFalse(portsIter.hasNext());
 
-
 		Throwable error = null;
 
 		// try some wrong values: String
-		try { NetUtils.getPortRangeFromString("localhost"); } catch(Throwable t) { error = t; }
+		try {
+			NetUtils.getPortRangeFromString("localhost");
+		} catch (Throwable t) {
+			error = t;
+		}
 		Assert.assertTrue(error instanceof NumberFormatException);
 		error = null;
 
 		// incomplete range
-		try { NetUtils.getPortRangeFromString("5-"); } catch(Throwable t) { error = t; }
+		try {
+			NetUtils.getPortRangeFromString("5-");
+		} catch (Throwable t) {
+			error = t;
+		}
 		Assert.assertTrue(error instanceof NumberFormatException);
 		error = null;
 
 		// incomplete range
-		try { NetUtils.getPortRangeFromString("-5"); } catch(Throwable t) { error = t; }
+		try {
+			NetUtils.getPortRangeFromString("-5");
+		} catch (Throwable t) {
+			error = t;
+		}
 		Assert.assertTrue(error instanceof NumberFormatException);
 		error = null;
 
 		// empty range
-		try { NetUtils.getPortRangeFromString(",5"); } catch(Throwable t) { error = t; }
+		try {
+			NetUtils.getPortRangeFromString(",5");
+		} catch (Throwable t) {
+			error = t;
+		}
 		Assert.assertTrue(error instanceof NumberFormatException);
-		error = null;
-
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java b/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java
index edaeb7a..a891777 100644
--- a/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/NumberSequenceIteratorTest.java
@@ -18,12 +18,14 @@
 
 package org.apache.flink.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+/**
+ * Tests for the {@link NumberSequenceIterator}.
+ */
 public class NumberSequenceIteratorTest extends TestLogger {
 
 	@Test
@@ -33,14 +35,14 @@ public class NumberSequenceIteratorTest extends TestLogger {
 		testSplitting(new NumberSequenceIterator(-100, 0), 5);
 		testSplitting(new NumberSequenceIterator(-100, 100), 3);
 	}
-	
+
 	@Test
 	public void testSplittingLargeRangesBy2() {
 		testSplitting(new NumberSequenceIterator(0, Long.MAX_VALUE), 2);
 		testSplitting(new NumberSequenceIterator(-1000000000L, Long.MAX_VALUE), 2);
 		testSplitting(new NumberSequenceIterator(Long.MIN_VALUE, Long.MAX_VALUE), 2);
 	}
-	
+
 	@Test
 	public void testSplittingTooSmallRanges() {
 		testSplitting(new NumberSequenceIterator(0, 0), 2);
@@ -48,29 +50,28 @@ public class NumberSequenceIteratorTest extends TestLogger {
 		testSplitting(new NumberSequenceIterator(-5, -4), 3);
 		testSplitting(new NumberSequenceIterator(10, 15), 10);
 	}
-	
-	private static final void testSplitting(NumberSequenceIterator iter, int numSplits) {
+
+	private static void testSplitting(NumberSequenceIterator iter, int numSplits) {
 		NumberSequenceIterator[] splits = iter.split(numSplits);
-		
+
 		assertEquals(numSplits, splits.length);
-		
+
 		// test start and end of range
 		assertEquals(iter.getCurrent(), splits[0].getCurrent());
-		assertEquals(iter.getTo(), splits[numSplits-1].getTo());
-		
+		assertEquals(iter.getTo(), splits[numSplits - 1].getTo());
+
 		// test continuous range
 		for (int i = 1; i < splits.length; i++) {
-			assertEquals(splits[i-1].getTo() + 1, splits[i].getCurrent());
+			assertEquals(splits[i - 1].getTo() + 1, splits[i].getCurrent());
 		}
-		
+
 		testMaxSplitDiff(splits);
 	}
-	
-	
-	private static final void testMaxSplitDiff(NumberSequenceIterator[] iters) {
+
+	private static void testMaxSplitDiff(NumberSequenceIterator[] iters) {
 		long minSplitSize = Long.MAX_VALUE;
 		long maxSplitSize = Long.MIN_VALUE;
-		
+
 		for (NumberSequenceIterator iter : iters) {
 			long diff;
 			if (iter.getTo() < iter.getCurrent()) {
@@ -81,12 +82,12 @@ public class NumberSequenceIteratorTest extends TestLogger {
 			if (diff < 0) {
 				diff = Long.MAX_VALUE;
 			}
-			
+
 			minSplitSize = Math.min(minSplitSize, diff);
 			maxSplitSize = Math.max(maxSplitSize, diff);
 		}
-		
-		assertTrue(maxSplitSize == minSplitSize || maxSplitSize-1 == minSplitSize);
+
+		assertTrue(maxSplitSize == minSplitSize || maxSplitSize - 1 == minSplitSize);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/OutputTagTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/OutputTagTest.java b/flink-core/src/test/java/org/apache/flink/util/OutputTagTest.java
index 1caa5b2..65f2bb6 100644
--- a/flink-core/src/test/java/org/apache/flink/util/OutputTagTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/OutputTagTest.java
@@ -15,11 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.util;
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+
 import org.junit.Test;
 
+/**
+ * Tests for the {@link OutputTag}.
+ */
 public class OutputTagTest {
 
 	@Test(expected = NullPointerException.class)

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java b/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java
index ade8d0f..c605bab 100644
--- a/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java
@@ -24,6 +24,9 @@ import org.junit.Test;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+/**
+ * Tests for the {@link ResourceGuard}.
+ */
 public class ResourceGuardTest extends TestLogger {
 
 	@Test
@@ -128,20 +131,20 @@ public class ResourceGuardTest extends TestLogger {
 	@Test
 	public void testLeaseCloseIsIdempotent() throws Exception {
 		ResourceGuard resourceGuard = new ResourceGuard();
-		ResourceGuard.Lease lease_1 = resourceGuard.acquireResource();
-		ResourceGuard.Lease lease_2 = resourceGuard.acquireResource();
+		ResourceGuard.Lease lease1 = resourceGuard.acquireResource();
+		ResourceGuard.Lease lease2 = resourceGuard.acquireResource();
 		Assert.assertEquals(2, resourceGuard.getLeaseCount());
-		lease_1.close();
+		lease1.close();
 		Assert.assertEquals(1, resourceGuard.getLeaseCount());
-		lease_1.close();
+		lease1.close();
 		Assert.assertEquals(1, resourceGuard.getLeaseCount());
-		lease_2.close();
+		lease2.close();
 		Assert.assertEquals(0, resourceGuard.getLeaseCount());
-		ResourceGuard.Lease lease_3 = resourceGuard.acquireResource();
+		ResourceGuard.Lease lease3 = resourceGuard.acquireResource();
 		Assert.assertEquals(1, resourceGuard.getLeaseCount());
-		lease_2.close();
+		lease2.close();
 		Assert.assertEquals(1, resourceGuard.getLeaseCount());
-		lease_3.close();
+		lease3.close();
 		Assert.assertEquals(0, resourceGuard.getLeaseCount());
 		resourceGuard.close();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/SerializedValueTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/SerializedValueTest.java b/flink-core/src/test/java/org/apache/flink/util/SerializedValueTest.java
index fda368a..2c758e6 100644
--- a/flink-core/src/test/java/org/apache/flink/util/SerializedValueTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/SerializedValueTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.util;
 
 import org.apache.flink.core.testutils.CommonTestUtils;
+
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -26,6 +27,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
+/**
+ * Tests for the {@link SerializedValue}.
+ */
 public class SerializedValueTest {
 
 	@Test
@@ -33,7 +37,7 @@ public class SerializedValueTest {
 		try {
 			final String value = "teststring";
 
-			SerializedValue<String> v = new SerializedValue<String>(value);
+			SerializedValue<String> v = new SerializedValue<>(value);
 			SerializedValue<String> copy = CommonTestUtils.createCopySerializable(v);
 
 			assertEquals(value, v.deserializeValue(getClass().getClassLoader()));
@@ -55,7 +59,7 @@ public class SerializedValueTest {
 	@Test
 	public void testNullValue() {
 		try {
-			SerializedValue<Object> v = new SerializedValue<Object>(null);
+			SerializedValue<Object> v = new SerializedValue<>(null);
 			SerializedValue<Object> copy = CommonTestUtils.createCopySerializable(v);
 
 			assertNull(copy.deserializeValue(getClass().getClassLoader()));

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
index ca28e95..5f705b4 100644
--- a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
@@ -18,12 +18,14 @@
 
 package org.apache.flink.util;
 
+import org.junit.Test;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
-import org.apache.flink.util.StringUtils;
-import org.junit.Test;
-
+/**
+ * Tests for the {@link StringUtils}.
+ */
 public class StringUtilsTest extends TestLogger {
 
 	@Test
@@ -32,11 +34,11 @@ public class StringUtilsTest extends TestLogger {
 		String controlString = StringUtils.showControlCharacters(testString);
 		assertEquals("\\b \\t \\n \\f \\r default", controlString);
 	}
-	
+
 	@Test
 	public void testArrayToString() {
 		double[] array = {1.0};
-		String controlString = StringUtils.arrayToString(array );
+		String controlString = StringUtils.arrayToString(array);
 		assertEquals("[1.0]", controlString);
 	}
 
@@ -47,7 +49,7 @@ public class StringUtilsTest extends TestLogger {
 		byte[] expectedArray = new byte[]{1, -97, 49, 74 };
 		assertArrayEquals(expectedArray, hexArray);
 	}
-	
+
 	@Test
 	public void testHexArrayToString() {
 		byte[] byteArray = new byte[]{1, -97, 49, 74 };

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/StringValueUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/StringValueUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/StringValueUtilsTest.java
index 6b15196..f147725 100644
--- a/flink-core/src/test/java/org/apache/flink/util/StringValueUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/StringValueUtilsTest.java
@@ -18,15 +18,15 @@
 
 package org.apache.flink.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.StringValueUtils.WhitespaceTokenizer;
 
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Tests for the {@link StringValueUtils}.
  */
@@ -38,14 +38,14 @@ public class StringValueUtilsTest extends TestLogger {
 		StringValueUtils.toLowerCase(testString);
 		assertEquals(new StringValue("test"), testString);
 	}
-	
+
 	@Test
 	public void testReplaceNonWordChars() {
 		StringValue testString = new StringValue("TEST123_@");
 		StringValueUtils.replaceNonWordChars(testString, '!');
 		assertEquals(new StringValue("TEST123_!"), testString);
 	}
-	
+
 	@Test
 	public void testTokenizerOnStringWithoutNexToken() {
 		StringValue testString = new StringValue("test");
@@ -56,7 +56,7 @@ public class StringValueUtilsTest extends TestLogger {
 		//next token is not exist
 		assertFalse(tokenizer.next(testString));
 	}
-	
+
 	@Test
 	public void testTokenizerOnStringWithNexToken() {
 		StringValue testString = new StringValue("test test");
@@ -64,7 +64,7 @@ public class StringValueUtilsTest extends TestLogger {
 		tokenizer.setStringToTokenize(testString);
 		assertTrue(tokenizer.next(testString));
 	}
-	
+
 	@Test
 	public void testTokenizerOnStringOnlyWithDelimiter() {
 		StringValue testString = new StringValue("    ");

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/test/java/org/apache/flink/util/UnionIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/UnionIteratorTest.java b/flink-core/src/test/java/org/apache/flink/util/UnionIteratorTest.java
index 4c1fc41..9039412 100644
--- a/flink-core/src/test/java/org/apache/flink/util/UnionIteratorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/UnionIteratorTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.util;
 
-import org.apache.flink.util.TraversableOnceException;
-import org.apache.flink.util.UnionIterator;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -27,8 +25,14 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
+/**
+ * Tests for the {@link UnionIterator}.
+ */
 public class UnionIteratorTest {
 
 	@Test
@@ -40,7 +44,7 @@ public class UnionIteratorTest {
 			assertFalse(iter.iterator().hasNext());
 
 			iter.clear();
-			
+
 			try {
 				iter.iterator().next();
 				fail("should fail with an exception");
@@ -52,7 +56,7 @@ public class UnionIteratorTest {
 			iter.addList(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
 			iter.addList(Collections.<Integer>emptyList());
 			iter.addList(Arrays.asList(8, 9, 10, 11));
-			
+
 			int val = 1;
 			for (int i : iter) {
 				assertEquals(val++, i);
@@ -63,15 +67,15 @@ public class UnionIteratorTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testTraversableOnce() {
 		try {
 			UnionIterator<Integer> iter = new UnionIterator<>();
-			
+
 			// should succeed
 			iter.iterator();
-			
+
 			// should fail
 			try {
 				iter.iterator();
@@ -113,13 +117,13 @@ public class UnionIteratorTest {
 			// reset the thing, add some data
 			iter.clear();
 			iter.addList(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-			
+
 			// should succeed
 			Iterator<Integer> ints = iter.iterator();
 			assertNotNull(ints.next());
 			assertNotNull(ints.next());
 			assertNotNull(ints.next());
-			
+
 			// should fail if called in the middle of operations
 			try {
 				iter.iterator();
@@ -133,7 +137,7 @@ public class UnionIteratorTest {
 
 			// should succeed again
 			assertFalse(iter.iterator().hasNext());
-			
+
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/tools/maven/suppressions-core.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions-core.xml b/tools/maven/suppressions-core.xml
index 78341c9..9640815 100644
--- a/tools/maven/suppressions-core.xml
+++ b/tools/maven/suppressions-core.xml
@@ -131,9 +131,6 @@ under the License.
 		files="(.*)test[/\\](.*)types[/\\](.*)"
 		checks="AvoidStarImport|NeedBraces"/>
 
-	<suppress
-		files="(.*)util[/\\](.*)"
-		checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
 	<!--Only additional checks for test sources. Those checks were present in the "pre-strict" checkstyle but were not applied to test sources. We do not want to suppress them for sources directory-->
 	<suppress
 		files="(.*)test[/\\](.*)util[/\\](.*)"


[09/17] flink git commit: [hotfix] [core] Fix broken JavaDoc links in ConfigConstants

Posted by se...@apache.org.
[hotfix] [core] Fix broken JavaDoc links in ConfigConstants


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

Branch: refs/heads/master
Commit: 0030d6ab21197077438ba05654a5af353bc1acb7
Parents: e52db8b
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 25 17:32:17 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    | 52 ++++++++++----------
 1 file changed, 26 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0030d6ab/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index d0dd499..d1dcef5 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -668,7 +668,7 @@ public final class ConfigConstants {
 	/**
 	 * The port for the runtime monitor web-frontend server.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_PORT} instead.
+	 * @deprecated Use {@link WebOptions#PORT} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_PORT_KEY = "jobmanager.web.port";
@@ -676,7 +676,7 @@ public final class ConfigConstants {
 	/**
 	 * Config parameter to override SSL support for the JobManager Web UI
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_SSL_ENABLED} instead.
+	 * @deprecated Use {@link WebOptions#SSL_ENABLED} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_SSL_ENABLED = "jobmanager.web.ssl.enabled";
@@ -684,7 +684,7 @@ public final class ConfigConstants {
 	/**
 	 * The config parameter defining the flink web directory to be used by the webmonitor.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_TMP_DIR} instead.
+	 * @deprecated Use {@link WebOptions#TMP_DIR} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_TMPDIR_KEY = "jobmanager.web.tmpdir";
@@ -693,7 +693,7 @@ public final class ConfigConstants {
 	 * The config parameter defining the directory for uploading the job jars. If not specified a dynamic directory
 	 * will be used under the directory specified by JOB_MANAGER_WEB_TMPDIR_KEY.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_UPLOAD_DIR} instead.
+	 * @deprecated Use {@link WebOptions#UPLOAD_DIR} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_UPLOAD_DIR_KEY = "jobmanager.web.upload.dir";
@@ -701,7 +701,7 @@ public final class ConfigConstants {
 	/**
 	 * The config parameter defining the number of archived jobs for the jobmanager
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_ARCHIVE_COUNT} instead.
+	 * @deprecated Use {@link WebOptions#ARCHIVE_COUNT} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_ARCHIVE_COUNT = "jobmanager.web.history";
@@ -709,7 +709,7 @@ public final class ConfigConstants {
 	/**
 	 * The log file location (may be in /log for standalone but under log directory when using YARN)
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_LOG_PATH} instead.
+	 * @deprecated Use {@link WebOptions#LOG_PATH} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
@@ -717,7 +717,7 @@ public final class ConfigConstants {
 	/**
 	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_SUBMIT_ENABLE} instead.
+	 * @deprecated Use {@link WebOptions#SUBMIT_ENABLE} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_SUBMIT_ENABLED_KEY = "jobmanager.web.submit.enable";
@@ -733,7 +733,7 @@ public final class ConfigConstants {
 	/**
 	 * Config parameter defining the number of checkpoints to remember for recent history.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_CHECKPOINTS_HISTORY_SIZE} instead.
+	 * @deprecated Use {@link WebOptions#CHECKPOINTS_HISTORY_SIZE} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE = "jobmanager.web.checkpoints.history";
@@ -741,7 +741,7 @@ public final class ConfigConstants {
 	/**
 	 * Time after which cached stats are cleaned up if not accessed.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_BACKPRESSURE_CLEANUP_INTERVAL} instead.
+	 * @deprecated Use {@link WebOptions#BACKPRESSURE_CLEANUP_INTERVAL} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL = "jobmanager.web.backpressure.cleanup-interval";
@@ -749,7 +749,7 @@ public final class ConfigConstants {
 	/**
 	 * Time after which available stats are deprecated and need to be refreshed (by resampling).
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_BACKPRESSURE_REFRESH_INTERVAL} instead.
+	 * @deprecated Use {@link WebOptions#BACKPRESSURE_REFRESH_INTERVAL} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL = "jobmanager.web.backpressure.refresh-interval";
@@ -757,7 +757,7 @@ public final class ConfigConstants {
 	/**
 	 * Number of stack trace samples to take to determine back pressure.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_BACKPRESSURE_NUM_SAMPLES} instead.
+	 * @deprecated Use {@link WebOptions#BACKPRESSURE_NUM_SAMPLES} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES = "jobmanager.web.backpressure.num-samples";
@@ -765,7 +765,7 @@ public final class ConfigConstants {
 	/**
 	 * Delay between stack trace samples to determine back pressure.
 	 *
-	 * @deprecated Use {@link JobManagerOptions#WEB_BACKPRESSURE_DELAY} instead.
+	 * @deprecated Use {@link WebOptions#BACKPRESSURE_DELAY} instead.
 	 */
 	@Deprecated
 	public static final String JOB_MANAGER_WEB_BACK_PRESSURE_DELAY = "jobmanager.web.backpressure.delay-between-samples";
@@ -961,7 +961,7 @@ public final class ConfigConstants {
 	/**
 	 * State backend for checkpoints
 	 * 
-	 * @deprecated Use {@link CoreOptions#STATE_BACKEND} instead.
+	 * @deprecated Use {@link CheckpointingOptions#STATE_BACKEND} instead.
 	 */
 	@Deprecated
 	public static final String STATE_BACKEND = "state.backend";
@@ -1231,7 +1231,7 @@ public final class ConfigConstants {
 
 	/**
 	 * The default directory for savepoints.
-	 * @deprecated Use {@link CoreOptions#SAVEPOINT_DIRECTORY} instead.
+	 * @deprecated Use {@link CheckpointingOptions#SAVEPOINT_DIRECTORY} instead.
 	 */
 	@PublicEvolving
 	@Deprecated
@@ -1239,7 +1239,7 @@ public final class ConfigConstants {
 
 	/**
 	 * The default directory used for persistent checkpoints.
-	 * @deprecated Use {@link CoreOptions#CHECKPOINTS_DIRECTORY} instead.
+	 * @deprecated Use {@link CheckpointingOptions#CHECKPOINTS_DIRECTORY} instead.
 	 */
 	@PublicEvolving
 	@Deprecated
@@ -1248,7 +1248,7 @@ public final class ConfigConstants {
 	/**
 	 * @deprecated This key was used in Flink versions <= 1.1.X with the savepoint backend
 	 * configuration. We now always use the FileSystem for savepoints. For this,
-	 * the only relevant config key is {@link CoreOptions#SAVEPOINT_DIRECTORY}.
+	 * the only relevant config key is {@link CheckpointingOptions#SAVEPOINT_DIRECTORY}.
 	 */
 	@Deprecated
 	public static final String SAVEPOINT_FS_DIRECTORY_KEY = "savepoints.state.backend.fs.dir";
@@ -1560,7 +1560,7 @@ public final class ConfigConstants {
 	/**
 	 * The config key for the address of the JobManager web frontend.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_FRONTEND_ADDRESS} instead
+	 * @deprecated use {@link WebOptions#ADDRESS} instead
 	 */
 	@Deprecated
 	public static final ConfigOption<String> DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS =
@@ -1571,7 +1571,7 @@ public final class ConfigConstants {
 	 * The config key for the port of the JobManager web frontend.
 	 * Setting this value to {@code -1} disables the web frontend.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_PORT} instead
+	 * @deprecated use {@link WebOptions#PORT} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT = 8081;
@@ -1579,7 +1579,7 @@ public final class ConfigConstants {
 	/**
 	 * Default value to override SSL support for the JobManager web UI
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_SSL_ENABLED} instead
+	 * @deprecated use {@link WebOptions#SSL_ENABLED} instead
 	 */
 	@Deprecated
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED = true;
@@ -1587,7 +1587,7 @@ public final class ConfigConstants {
 	/**
 	 * The default number of archived jobs for the jobmanager
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_ARCHIVE_COUNT} instead
+	 * @deprecated use {@link WebOptions#ARCHIVE_COUNT} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
@@ -1595,7 +1595,7 @@ public final class ConfigConstants {
 	/**
 	 * By default, submitting jobs from the web-frontend is allowed.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_SUBMIT_ENABLE} instead
+	 * @deprecated use {@link WebOptions#SUBMIT_ENABLE} instead
 	 */
 	@Deprecated
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMIT_ENABLED = true;
@@ -1607,7 +1607,7 @@ public final class ConfigConstants {
 	/**
 	 * Default number of checkpoints to remember for recent history.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_CHECKPOINTS_HISTORY_SIZE} instead
+	 * @deprecated use {@link WebOptions#CHECKPOINTS_HISTORY_SIZE} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE = 10;
@@ -1615,7 +1615,7 @@ public final class ConfigConstants {
 	/**
 	 * Time after which cached stats are cleaned up.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_CLEANUP_INTERVAL} instead
+	 * @deprecated use {@link WebOptions#BACKPRESSURE_CLEANUP_INTERVAL} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL = 10 * 60 * 1000;
@@ -1623,7 +1623,7 @@ public final class ConfigConstants {
 	/**
 	 * Time after which available stats are deprecated and need to be refreshed (by resampling).
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_REFRESH_INTERVAL} instead
+	 * @deprecated use {@link WebOptions#BACKPRESSURE_REFRESH_INTERVAL} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL = 60 * 1000;
@@ -1631,7 +1631,7 @@ public final class ConfigConstants {
 	/**
 	 * Number of samples to take to determine back pressure.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_NUM_SAMPLES} instead
+	 * @deprecated use {@link WebOptions#BACKPRESSURE_NUM_SAMPLES} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES = 100;
@@ -1639,7 +1639,7 @@ public final class ConfigConstants {
 	/**
 	 * Delay between samples to determine back pressure.
 	 *
-	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_DELAY} instead
+	 * @deprecated use {@link WebOptions#BACKPRESSURE_DELAY} instead
 	 */
 	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAY = 50;


[07/17] flink git commit: [FLINK-5823] [checkpoints] State backends define checkpoint and savepoint directories, improved configuration

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 5893d1d..6a84a11 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
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 
 import org.junit.Test;
@@ -102,7 +103,7 @@ public class ExecutionGraphCheckpointCoordinatorTest {
 				counter,
 				store,
 				null,
-				null,
+				new MemoryStateBackend(),
 				CheckpointStatsTrackerTest.createTestTracker());
 
 		JobVertex jobVertex = new JobVertex("MockVertex");

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 7b9d9aa..3fe8613 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
@@ -43,6 +43,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
@@ -66,8 +67,6 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
 public class ArchivedExecutionGraphTest extends TestLogger {
-	private static JobVertexID v1ID = new JobVertexID();
-	private static JobVertexID v2ID = new JobVertexID();
 
 	private static ExecutionGraph runtimeGraph;
 
@@ -77,8 +76,8 @@ public class ArchivedExecutionGraphTest extends TestLogger {
 		// Setup
 		// -------------------------------------------------------------------------------------------------------------
 
-		v1ID = new JobVertexID();
-		v2ID = new JobVertexID();
+		JobVertexID v1ID = new JobVertexID();
+		JobVertexID v2ID = new JobVertexID();
 
 		JobVertex v1 = new JobVertex("v1", v1ID);
 		JobVertex v2 = new JobVertex("v2", v2ID);
@@ -89,7 +88,7 @@ public class ArchivedExecutionGraphTest extends TestLogger {
 		v1.setInvokableClass(AbstractInvokable.class);
 		v2.setInvokableClass(AbstractInvokable.class);
 
-		List<JobVertex> vertices = new ArrayList<JobVertex>(Arrays.asList(v1, v2));
+		List<JobVertex> vertices = new ArrayList<>(Arrays.asList(v1, v2));
 
 		ExecutionConfig config = new ExecutionConfig();
 
@@ -135,7 +134,7 @@ public class ArchivedExecutionGraphTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
-			null,
+			new MemoryStateBackend(),
 			statsTracker);
 
 		Map<String, Accumulator<?, ?>> userAccumulators = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 01d2346..452afd6 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
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
@@ -149,16 +150,16 @@ import static org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_AS
 import static org.apache.flink.runtime.testingUtils.TestingUtils.TESTING_TIMEOUT;
 import static org.apache.flink.runtime.testingUtils.TestingUtils.startTestingCluster;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import static org.mockito.Mockito.mock;
 
 public class JobManagerTest extends TestLogger {
 
 	@Rule
-	public TemporaryFolder tmpFolder = new TemporaryFolder();
+	public final TemporaryFolder tmpFolder = new TemporaryFolder();
 
 	private static ActorSystem system;
 
@@ -830,7 +831,7 @@ public class JobManagerTest extends TestLogger {
 
 		FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
 		Configuration config = new Configuration();
-		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.toURI().toString());
 
 		ActorSystem actorSystem = null;
 		ActorGateway jobManager = null;
@@ -932,13 +933,13 @@ public class JobManagerTest extends TestLogger {
 			}
 
 			// Verify savepoint path
-			assertNotEquals("Savepoint not triggered", null, savepointPath);
+			assertNotNull("Savepoint not triggered", savepointPath);
 
 			// Wait for job status change
 			Await.ready(cancelled, timeout);
 
-			File savepointFile = new File(savepointPath);
-			assertEquals(true, savepointFile.exists());
+			File savepointFile = new File(new Path(savepointPath).getPath());
+			assertTrue(savepointFile.exists());
 		} finally {
 			if (actorSystem != null) {
 				actorSystem.shutdown();
@@ -1157,7 +1158,7 @@ public class JobManagerTest extends TestLogger {
 
 		FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
 		Configuration config = new Configuration();
-		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.getAbsolutePath());
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, defaultSavepointDir.toURI().toString());
 
 		ActorSystem actorSystem = null;
 		ActorGateway jobManager = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
index 8ad368b..3d43cd8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.FutureUtil;
+import org.apache.flink.util.TernaryBoolean;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -93,7 +94,7 @@ public class MemoryStateBackendTest extends StateBackendTestBase<MemoryStateBack
 	@Test
 	public void testOversizedState() {
 		try {
-			MemoryStateBackend backend = new MemoryStateBackend(10);
+			MemoryStateBackend backend = new MemoryStateBackend(null, null, 10, TernaryBoolean.TRUE);
 			CheckpointStreamFactory streamFactory = backend.createStreamFactory(new JobID(), "test_op");
 
 			HashMap<String, Integer> state = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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
index a64faf1..cd4e355 100644
--- 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
@@ -18,22 +18,27 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackendFactory;
 import org.apache.flink.util.DynamicCodeLoadingException;
 
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
 
 import java.io.IOException;
+import java.net.URI;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -48,61 +53,199 @@ public class StateBackendLoadingTest {
 
 	private final ClassLoader cl = getClass().getClassLoader();
 
-	private final String backendKey = CoreOptions.STATE_BACKEND.key();
+	private final String backendKey = CheckpointingOptions.STATE_BACKEND.key();
 
 	// ------------------------------------------------------------------------
+	//  defaults
+	// ------------------------------------------------------------------------
 
 	@Test
 	public void testNoStateBackendDefined() throws Exception {
-		assertNull(AbstractStateBackend.loadStateBackendFromConfig(new Configuration(), cl, null));
+		assertNull(StateBackendLoader.loadStateBackendFromConfig(new Configuration(), cl, null));
 	}
 
 	@Test
 	public void testInstantiateMemoryBackendByDefault() throws Exception {
-		StateBackend backend = AbstractStateBackend
-				.loadStateBackendFromConfigOrCreateDefault(new Configuration(), cl, null);
+		StateBackend backend =
+				StateBackendLoader.fromApplicationOrConfigOrDefault(null, 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 
+	public void testApplicationDefinedHasPrecedence() throws Exception {
+		final StateBackend appBackend = Mockito.mock(StateBackend.class);
+
 		final Configuration config = new Configuration();
 		config.setString(backendKey, "jobmanager");
 
-		StateBackend backend = AbstractStateBackend
-				.loadStateBackendFromConfigOrCreateDefault(new Configuration(), cl, null);
+		StateBackend backend = StateBackendLoader.fromApplicationOrConfigOrDefault(appBackend, config, cl, null);
+		assertEquals(appBackend, backend);
+	}
 
-		assertTrue(backend instanceof MemoryStateBackend);
+	// ------------------------------------------------------------------------
+	//  Memory State Backend
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Validates loading a memory state backend from the cluster configuration.
+	 */
+	@Test
+	public void testLoadMemoryStateBackendNoParameters() 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 config1 = new Configuration();
+		config1.setString(backendKey, "jobmanager");
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, MemoryStateBackendFactory.class.getName());
+
+		StateBackend backend1 = StateBackendLoader.loadStateBackendFromConfig(config1, cl, null);
+		StateBackend backend2 = StateBackendLoader.loadStateBackendFromConfig(config2, cl, null);
+
+		assertTrue(backend1 instanceof MemoryStateBackend);
+		assertTrue(backend2 instanceof MemoryStateBackend);
 	}
 
+	/**
+	 * Validates loading a memory state backend with additional parameters from the cluster configuration.
+	 */
+	@Test
+	public void testLoadMemoryStateWithParameters() throws Exception {
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+		final Path expectedCheckpointPath = new Path(checkpointDir);
+		final Path expectedSavepointPath = new Path(savepointDir);
+
+		final boolean async = !CheckpointingOptions.ASYNC_SNAPSHOTS.defaultValue();
+
+		// 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, "jobmanager");
+		config1.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config1.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config1.setBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS, async);
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, MemoryStateBackendFactory.class.getName());
+		config2.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config2.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config2.setBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS, async);
+
+		MemoryStateBackend backend1 = (MemoryStateBackend)
+				StateBackendLoader.loadStateBackendFromConfig(config1, cl, null);
+		MemoryStateBackend backend2 = (MemoryStateBackend)
+				StateBackendLoader.loadStateBackendFromConfig(config2, cl, null);
+
+		assertNotNull(backend1);
+		assertNotNull(backend2);
+
+		assertEquals(expectedCheckpointPath, backend1.getCheckpointPath());
+		assertEquals(expectedCheckpointPath, backend2.getCheckpointPath());
+		assertEquals(expectedSavepointPath, backend1.getSavepointPath());
+		assertEquals(expectedSavepointPath, backend2.getSavepointPath());
+		assertEquals(async, backend1.isUsingAsynchronousSnapshots());
+		assertEquals(async, backend2.isUsingAsynchronousSnapshots());
+	}
+
+	/**
+	 * Validates taking the application-defined memory state backend and adding additional
+	 * parameters from the cluster configuration.
+	 */
+	@Test
+	public void testConfigureMemoryStateBackend() throws Exception {
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+		final Path expectedCheckpointPath = new Path(checkpointDir);
+		final Path expectedSavepointPath = new Path(savepointDir);
+
+		final int maxSize = 100;
+		final boolean async = !CheckpointingOptions.ASYNC_SNAPSHOTS.defaultValue();
+
+		final MemoryStateBackend backend = new MemoryStateBackend(maxSize, async);
+
+		final Configuration config = new Configuration();
+		config.setString(backendKey, "filesystem"); // check that this is not accidentally picked up
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config.setBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS, !async);
+
+		StateBackend loadedBackend = StateBackendLoader.fromApplicationOrConfigOrDefault(backend, config, cl, null);
+		assertTrue(loadedBackend instanceof MemoryStateBackend);
+
+		final MemoryStateBackend memBackend = (MemoryStateBackend) loadedBackend;
+		assertEquals(expectedCheckpointPath, memBackend.getCheckpointPath());
+		assertEquals(expectedSavepointPath, memBackend.getSavepointPath());
+		assertEquals(maxSize, memBackend.getMaxStateSize());
+		assertEquals(async, memBackend.isUsingAsynchronousSnapshots());
+	}
+
+	/**
+	 * Validates taking the application-defined memory state backend and adding additional
+	 * parameters from the cluster configuration, but giving precedence to application-defined
+	 * parameters over configuration-defined parameters.
+	 */
+	@Test
+	public void testConfigureMemoryStateBackendMixed() throws Exception {
+		final String appCheckpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+
+		final Path expectedCheckpointPath = new Path(appCheckpointDir);
+		final Path expectedSavepointPath = new Path(savepointDir);
+
+		final MemoryStateBackend backend = new MemoryStateBackend(appCheckpointDir, null);
+
+		final Configuration config = new Configuration();
+		config.setString(backendKey, "filesystem"); // check that this is not accidentally picked up
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); // this parameter should not be picked up
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+
+		StateBackend loadedBackend = StateBackendLoader.fromApplicationOrConfigOrDefault(backend, config, cl, null);
+		assertTrue(loadedBackend instanceof MemoryStateBackend);
+
+		final MemoryStateBackend memBackend = (MemoryStateBackend) loadedBackend;
+		assertEquals(expectedCheckpointPath, memBackend.getCheckpointPath());
+		assertEquals(expectedSavepointPath, memBackend.getSavepointPath());
+	}
+
+	// ------------------------------------------------------------------------
+	//  File System State Backend
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Validates loading a file system state backend with additional parameters from the cluster configuration.
+	 */
 	@Test
 	public void testLoadFileSystemStateBackend() throws Exception {
-		final String checkpointDir = new Path(tmp.getRoot().toURI()).toString();
-		final Path expectedPath = new Path(checkpointDir);
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+		final Path expectedCheckpointsPath = new Path(checkpointDir);
+		final Path expectedSavepointsPath = new Path(savepointDir);
 		final int threshold = 1000000;
+		final boolean async = !CheckpointingOptions.ASYNC_SNAPSHOTS.defaultValue();
 
 		// 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);
+		config1.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config1.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config1.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, threshold);
+		config1.setBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS, async);
 
 		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);
+		config2.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config2.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config2.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, threshold);
+		config2.setBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS, async);
 
-		StateBackend backend2 = AbstractStateBackend
-				.loadStateBackendFromConfigOrCreateDefault(config2, cl, null);
+		StateBackend backend1 = StateBackendLoader.loadStateBackendFromConfig(config1, cl, null);
+		StateBackend backend2 = StateBackendLoader.loadStateBackendFromConfig(config2, cl, null);
 
 		assertTrue(backend1 instanceof FsStateBackend);
 		assertTrue(backend2 instanceof FsStateBackend);
@@ -110,13 +253,55 @@ public class StateBackendLoadingTest {
 		FsStateBackend fs1 = (FsStateBackend) backend1;
 		FsStateBackend fs2 = (FsStateBackend) backend2;
 
-		assertEquals(expectedPath, fs1.getBasePath());
-		assertEquals(expectedPath, fs2.getBasePath());
+		assertEquals(expectedCheckpointsPath, fs1.getCheckpointPath());
+		assertEquals(expectedCheckpointsPath, fs2.getCheckpointPath());
+		assertEquals(expectedSavepointsPath, fs1.getSavepointPath());
+		assertEquals(expectedSavepointsPath, fs2.getSavepointPath());
 		assertEquals(threshold, fs1.getMinFileSizeThreshold());
 		assertEquals(threshold, fs2.getMinFileSizeThreshold());
+		assertEquals(async, fs1.isUsingAsynchronousSnapshots());
+		assertEquals(async, fs2.isUsingAsynchronousSnapshots());
 	}
 
 	/**
+	 * Validates taking the application-defined file system state backend and adding with additional
+	 * parameters from the cluster configuration, but giving precedence to application-defined
+	 * parameters over configuration-defined parameters.
+	 */
+	@Test
+	public void testLoadFileSystemStateBackendMixed() throws Exception {
+		final String appCheckpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+
+		final Path expectedCheckpointsPath = new Path(new URI(appCheckpointDir));
+		final Path expectedSavepointsPath = new Path(savepointDir);
+
+		final int threshold = 1000000;
+
+		final FsStateBackend backend = new FsStateBackend(new URI(appCheckpointDir), threshold);
+
+		final Configuration config = new Configuration();
+		config.setString(backendKey, "jobmanager"); // this should not be picked up 
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); // this should not be picked up
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 20); // this should not be picked up
+
+		final StateBackend loadedBackend =
+				StateBackendLoader.fromApplicationOrConfigOrDefault(backend, config, cl, null);
+		assertTrue(loadedBackend instanceof FsStateBackend);
+
+		final FsStateBackend fs = (FsStateBackend) loadedBackend;
+		assertEquals(expectedCheckpointsPath, fs.getCheckpointPath());
+		assertEquals(expectedSavepointsPath, fs.getSavepointPath());
+		assertEquals(threshold, fs.getMinFileSizeThreshold());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Failures
+	// ------------------------------------------------------------------------
+
+	/**
 	 * This test makes sure that failures properly manifest when the state backend could not be loaded.
 	 */
 	@Test
@@ -126,7 +311,7 @@ public class StateBackendLoadingTest {
 		// 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);
+			StateBackendLoader.fromApplicationOrConfigOrDefault(null, config, cl, null);
 			fail("should fail with an exception");
 		} catch (DynamicCodeLoadingException ignored) {
 			// expected
@@ -135,7 +320,7 @@ public class StateBackendLoadingTest {
 		// try a class that is not a factory
 		config.setString(backendKey, java.io.File.class.getName());
 		try {
-			AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(config, cl, null);
+			StateBackendLoader.fromApplicationOrConfigOrDefault(null, config, cl, null);
 			fail("should fail with an exception");
 		} catch (DynamicCodeLoadingException ignored) {
 			// expected
@@ -144,7 +329,7 @@ public class StateBackendLoadingTest {
 		// a factory that fails
 		config.setString(backendKey, FailingFactory.class.getName());
 		try {
-			AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(config, cl, null);
+			StateBackendLoader.fromApplicationOrConfigOrDefault(null, config, cl, null);
 			fail("should fail with an exception");
 		} catch (IOException ignored) {
 			// expected
@@ -152,12 +337,103 @@ public class StateBackendLoadingTest {
 	}
 
 	// ------------------------------------------------------------------------
+	//  High-availability default
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This tests that in the case of configured high-availability, the memory state backend
+	 * automatically grabs the HA persistence directory.
+	 */
+	@Test
+	public void testHighAvailabilityDefaultFallback() throws Exception {
+		final String haPersistenceDir = new Path(tmp.newFolder().toURI()).toString();
+		final Path expectedCheckpointPath = new Path(haPersistenceDir);
+
+		final Configuration config1 = new Configuration();
+		config1.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config1.setString(HighAvailabilityOptions.HA_CLUSTER_ID, "myCluster");
+		config1.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haPersistenceDir);
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, "jobmanager");
+		config2.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config2.setString(HighAvailabilityOptions.HA_CLUSTER_ID, "myCluster");
+		config2.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haPersistenceDir);
+
+		final MemoryStateBackend appBackend = new MemoryStateBackend();
+
+		final StateBackend loaded1 = StateBackendLoader.fromApplicationOrConfigOrDefault(appBackend, config1, cl, null);
+		final StateBackend loaded2 = StateBackendLoader.fromApplicationOrConfigOrDefault(null, config1, cl, null);
+		final StateBackend loaded3 = StateBackendLoader.fromApplicationOrConfigOrDefault(null, config2, cl, null);
+
+		assertTrue(loaded1 instanceof MemoryStateBackend);
+		assertTrue(loaded2 instanceof MemoryStateBackend);
+		assertTrue(loaded3 instanceof MemoryStateBackend);
+
+		final MemoryStateBackend memBackend1 = (MemoryStateBackend) loaded1;
+		final MemoryStateBackend memBackend2 = (MemoryStateBackend) loaded2;
+		final MemoryStateBackend memBackend3 = (MemoryStateBackend) loaded3;
+
+		assertNotNull(memBackend1.getCheckpointPath());
+		assertNotNull(memBackend2.getCheckpointPath());
+		assertNotNull(memBackend3.getCheckpointPath());
+		assertNull(memBackend1.getSavepointPath());
+		assertNull(memBackend2.getSavepointPath());
+		assertNull(memBackend3.getSavepointPath());
+
+		assertEquals(expectedCheckpointPath, memBackend1.getCheckpointPath().getParent());
+		assertEquals(expectedCheckpointPath, memBackend2.getCheckpointPath().getParent());
+		assertEquals(expectedCheckpointPath, memBackend3.getCheckpointPath().getParent());
+	}
+
+	@Test
+	public void testHighAvailabilityDefaultFallbackLocalPaths() throws Exception {
+		final String haPersistenceDir = new Path(tmp.newFolder().getAbsolutePath()).toString();
+		final Path expectedCheckpointPath = new Path(haPersistenceDir).makeQualified(FileSystem.getLocalFileSystem());
+
+		final Configuration config1 = new Configuration();
+		config1.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config1.setString(HighAvailabilityOptions.HA_CLUSTER_ID, "myCluster");
+		config1.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haPersistenceDir);
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, "jobmanager");
+		config2.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config2.setString(HighAvailabilityOptions.HA_CLUSTER_ID, "myCluster");
+		config2.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haPersistenceDir);
+
+		final MemoryStateBackend appBackend = new MemoryStateBackend();
+
+		final StateBackend loaded1 = StateBackendLoader.fromApplicationOrConfigOrDefault(appBackend, config1, cl, null);
+		final StateBackend loaded2 = StateBackendLoader.fromApplicationOrConfigOrDefault(null, config1, cl, null);
+		final StateBackend loaded3 = StateBackendLoader.fromApplicationOrConfigOrDefault(null, config2, cl, null);
+
+		assertTrue(loaded1 instanceof MemoryStateBackend);
+		assertTrue(loaded2 instanceof MemoryStateBackend);
+		assertTrue(loaded3 instanceof MemoryStateBackend);
+
+		final MemoryStateBackend memBackend1 = (MemoryStateBackend) loaded1;
+		final MemoryStateBackend memBackend2 = (MemoryStateBackend) loaded2;
+		final MemoryStateBackend memBackend3 = (MemoryStateBackend) loaded3;
+
+		assertNotNull(memBackend1.getCheckpointPath());
+		assertNotNull(memBackend2.getCheckpointPath());
+		assertNotNull(memBackend3.getCheckpointPath());
+		assertNull(memBackend1.getSavepointPath());
+		assertNull(memBackend2.getSavepointPath());
+		assertNull(memBackend3.getSavepointPath());
+
+		assertEquals(expectedCheckpointPath, memBackend1.getCheckpointPath().getParent());
+		assertEquals(expectedCheckpointPath, memBackend2.getCheckpointPath().getParent());
+		assertEquals(expectedCheckpointPath, memBackend3.getCheckpointPath().getParent());
+	}
+
+	// ------------------------------------------------------------------------
 
 	static final class FailingFactory implements StateBackendFactory<StateBackend> {
-		private static final long serialVersionUID = 1L;
 
 		@Override
-		public StateBackend createFromConfig(Configuration config) throws IllegalConfigurationException, IOException {
+		public StateBackend createFromConfig(Configuration config) throws IOException {
 			throw new IOException("fail!");
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
index 63563f3..ccf6baf 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.operators.util.CorruptConfigurationException;
-import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.util.ClassLoaderUtil;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.TimeCharacteristic;
@@ -442,7 +442,7 @@ public class StreamConfig implements Serializable {
 	//  State backend
 	// ------------------------------------------------------------------------
 
-	public void setStateBackend(AbstractStateBackend backend) {
+	public void setStateBackend(StateBackend backend) {
 		if (backend != null) {
 			try {
 				InstantiationUtil.writeObjectToConfig(backend, this.config, STATE_BACKEND);
@@ -452,7 +452,7 @@ public class StreamConfig implements Serializable {
 		}
 	}
 
-	public AbstractStateBackend getStateBackend(ClassLoader cl) {
+	public StateBackend getStateBackend(ClassLoader cl) {
 		try {
 			return InstantiationUtil.readObjectFromConfig(this.config, STATE_BACKEND, cl);
 		} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 a1bd355..f89b916 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
@@ -36,13 +36,13 @@ import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -734,19 +734,13 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 	// ------------------------------------------------------------------------
 
 	private StateBackend createStateBackend() throws Exception {
-		final StateBackend fromJob = configuration.getStateBackend(getUserCodeClassLoader());
+		final StateBackend fromApplication = configuration.getStateBackend(getUserCodeClassLoader());
 
-		if (fromJob != null) {
-			// backend has been configured on the environment
-			LOG.info("Using user-defined state backend: {}.", fromJob);
-			return fromJob;
-		}
-		else {
-			return AbstractStateBackend.loadStateBackendFromConfigOrCreateDefault(
-					getEnvironment().getTaskManagerInfo().getConfiguration(),
-					getUserCodeClassLoader(),
-					LOG);
-		}
+		return StateBackendLoader.fromApplicationOrConfigOrDefault(
+				fromApplication,
+				getEnvironment().getTaskManagerInfo().getConfiguration(),
+				getUserCodeClassLoader(),
+				LOG);
 	}
 
 	public OperatorStateBackend createOperatorStateBackend(

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
index bc69e1c..2051771 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java
@@ -291,6 +291,8 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 
 	private static class SyncFailureInducingStateBackend extends MemoryStateBackend {
 
+		private static final long serialVersionUID = -1915780414440060539L;
+
 		@Override
 		public OperatorStateBackend createOperatorStateBackend(Environment env, String operatorIdentifier) throws Exception {
 			return new DefaultOperatorStateBackend(
@@ -308,10 +310,18 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 				}
 			};
 		}
+
+		@Override
+		public SyncFailureInducingStateBackend configure(Configuration config) {
+			// retain this instance, no re-configuration!
+			return this;
+		}
 	}
 
 	private static class AsyncFailureInducingStateBackend extends MemoryStateBackend {
 
+		private static final long serialVersionUID = -7613628662587098470L;
+
 		@Override
 		public OperatorStateBackend createOperatorStateBackend(Environment env, String operatorIdentifier) throws Exception {
 			return new DefaultOperatorStateBackend(
@@ -334,6 +344,12 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 				}
 			};
 		}
+
+		@Override
+		public AsyncFailureInducingStateBackend configure(Configuration config) {
+			// retain this instance, no re-configuration!
+			return this;
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -356,6 +372,12 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 				new ExecutionConfig(),
 				true);
 		}
+
+		@Override
+		public LockingStreamStateBackend configure(Configuration config) {
+			// retain this instance, no re-configuration!
+			return this;
+		}
 	}
 
 	private static final class LockingOutputStreamFactory implements CheckpointStreamFactory {
@@ -373,7 +395,7 @@ public class TaskCheckpointingBehaviourTest extends TestLogger {
 
 		@Override
 		public StreamStateHandle closeAndGetHandle() throws IOException {
-			return null;
+			throw new UnsupportedOperationException();
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
index 99a9d1f..ffe220e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
@@ -32,15 +32,15 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.operators.testutils.MockEnvironment;
 import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamMap;
@@ -70,7 +70,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 
-import static org.apache.flink.runtime.state.filesystem.FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -85,9 +84,9 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	@Parameterized.Parameters(name = "StateBackend: {0}")
 	public static Collection<String> parameters () {
 		return Arrays.asList(
-			AbstractStateBackend.MEMORY_STATE_BACKEND_NAME,
-			AbstractStateBackend.FS_STATE_BACKEND_NAME,
-			AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME);
+				StateBackendLoader.MEMORY_STATE_BACKEND_NAME,
+				StateBackendLoader.FS_STATE_BACKEND_NAME,
+				StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME);
 	}
 
 	@ClassRule
@@ -97,9 +96,9 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 
 	public PojoSerializerUpgradeTest(String backendType) throws IOException, DynamicCodeLoadingException {
 		Configuration config = new Configuration();
-		config.setString(CoreOptions.STATE_BACKEND, backendType);
-		config.setString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, temporaryFolder.newFolder().toURI().toString());
-		stateBackend = AbstractStateBackend.loadStateBackendFromConfig(config, Thread.currentThread().getContextClassLoader(), null);
+		config.setString(CheckpointingOptions.STATE_BACKEND, backendType);
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, temporaryFolder.newFolder().toURI().toString());
+		stateBackend = StateBackendLoader.loadStateBackendFromConfig(config, Thread.currentThread().getContextClassLoader(), null);
 	}
 
 	private static final String POJO_NAME = "Pojo";


[12/17] flink git commit: [FLINK-5823] [checkpoints] Pass state backend to checkpoint coordinator

Posted by se...@apache.org.
[FLINK-5823] [checkpoints] Pass state backend to checkpoint coordinator


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

Branch: refs/heads/master
Commit: 7d820d6fe17341463b2a0f9cd1cea1ef085eed21
Parents: 0030d6a
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 25 13:23:46 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../checkpoint/CheckpointCoordinator.java       |  7 +++
 .../runtime/executiongraph/ExecutionGraph.java  |  3 +-
 .../executiongraph/ExecutionGraphBuilder.java   | 20 ++++-----
 ...tCoordinatorExternalizedCheckpointsTest.java |  8 ++--
 .../CheckpointCoordinatorFailureTest.java       |  2 +
 .../CheckpointCoordinatorMasterHooksTest.java   |  2 +
 .../checkpoint/CheckpointCoordinatorTest.java   | 31 ++++++++++++++
 .../checkpoint/CheckpointStateRestoreTest.java  |  4 ++
 .../environment/StreamExecutionEnvironment.java | 35 +++++++++------
 .../flink/streaming/api/graph/StreamGraph.java  |  8 ++--
 .../api/scala/StreamExecutionEnvironment.scala  | 45 ++++++++++++--------
 11 files changed, 118 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 a04e34e..40fa2bd 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
@@ -41,6 +41,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.SharedStateRegistryFactory;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StringUtils;
@@ -116,6 +117,10 @@ public class CheckpointCoordinator {
 	 * accessing this don't block the job manager actor and run asynchronously. */
 	private final CompletedCheckpointStore completedCheckpointStore;
 
+	/** The root checkpoint state backend, which is responsible for initializing the
+	 * checkpoint, storing the metadata, and cleaning up the checkpoint */
+	private final StateBackend checkpointStateBackend;
+
 	/** Default directory for persistent checkpoints; <code>null</code> if none configured.
 	 * THIS WILL BE REPLACED BY PROPER STATE-BACKEND METADATA WRITING */
 	@Nullable
@@ -196,6 +201,7 @@ public class CheckpointCoordinator {
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore completedCheckpointStore,
 			@Nullable String checkpointDirectory,
+			StateBackend checkpointStateBackend,
 			Executor executor,
 			SharedStateRegistryFactory sharedStateRegistryFactory) {
 
@@ -233,6 +239,7 @@ public class CheckpointCoordinator {
 		this.pendingCheckpoints = new LinkedHashMap<>();
 		this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
 		this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
+		this.checkpointStateBackend = checkNotNull(checkpointStateBackend);
 		this.checkpointDirectory = checkpointDirectory;
 		this.executor = checkNotNull(executor);
 		this.sharedStateRegistryFactory = checkNotNull(sharedStateRegistryFactory);

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 202839c..f88a6b62 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
@@ -452,7 +452,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore checkpointStore,
 			String checkpointDir,
-			StateBackend metadataStore,
+			StateBackend checkpointStateBackend,
 			CheckpointStatsTracker statsTracker) {
 
 		// simple sanity checks
@@ -482,6 +482,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			checkpointIDCounter,
 			checkpointStore,
 			checkpointDir,
+			checkpointStateBackend,
 			ioExecutor,
 			SharedStateRegistry.DEFAULT_FACTORY);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 47948a9..c742903 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
@@ -115,16 +115,16 @@ public class ExecutionGraphBuilder {
 		final ExecutionGraph executionGraph;
 		try {
 			executionGraph = (prior != null) ? prior :
-                new ExecutionGraph(
-                    jobInformation,
-                    futureExecutor,
-                    ioExecutor,
-                    timeout,
-                    restartStrategy,
-                    failoverStrategy,
-                    slotProvider,
-                    classLoader,
-                    blobWriter);
+				new ExecutionGraph(
+					jobInformation,
+					futureExecutor,
+					ioExecutor,
+					timeout,
+					restartStrategy,
+					failoverStrategy,
+					slotProvider,
+					classLoader,
+					blobWriter);
 		} catch (IOException e) {
 			throw new JobException("Could not create the ExecutionGraph.", e);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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
index edc29fe..f55e0d3 100644
--- 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
@@ -31,6 +31,7 @@ import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -54,7 +55,7 @@ import static org.junit.Assert.assertTrue;
 public class CheckpointCoordinatorExternalizedCheckpointsTest {
 
 	@Rule
-	public TemporaryFolder tmp = new TemporaryFolder();
+	public final TemporaryFolder tmp = new TemporaryFolder();
 
 	/**
 	 * Triggers multiple externalized checkpoints and verifies that the metadata
@@ -69,6 +70,7 @@ public class CheckpointCoordinatorExternalizedCheckpointsTest {
 			ExternalizedCheckpointSettings.externalizeCheckpoints(false);
 
 		final File checkpointDir = tmp.newFolder();
+		final FsStateBackend stateBackend = new FsStateBackend(checkpointDir.toURI());
 
 		// create some mock Execution vertices that receive the checkpoint trigger messages
 		final ExecutionAttemptID attemptID1 = new ExecutionAttemptID();
@@ -94,6 +96,7 @@ public class CheckpointCoordinatorExternalizedCheckpointsTest {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			checkpointDir.getAbsolutePath(),
+			stateBackend,
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -109,8 +112,7 @@ public class CheckpointCoordinatorExternalizedCheckpointsTest {
 
 			coord.triggerCheckpoint(timestamp1, false);
 
-			long checkpointId1 = coord.getPendingCheckpoints().entrySet().iterator().next()
-				.getKey();
+			long checkpointId1 = coord.getPendingCheckpoints().entrySet().iterator().next().getKey();
 
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId1));
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId1));

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 7c95a34..add7447 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
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
@@ -78,6 +79,7 @@ public class CheckpointCoordinatorFailureTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new FailingCompletedCheckpointStore(),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
index 2f860e0..e53bf3a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -402,6 +403,7 @@ public class CheckpointCoordinatorMasterHooksTest {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(10),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 2572bc1..609e91c 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
@@ -49,6 +49,7 @@ import org.apache.flink.runtime.state.StateHandleID;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore;
 import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare;
@@ -141,6 +142,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -202,6 +204,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -254,6 +257,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -307,6 +311,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -410,6 +415,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -530,6 +536,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -698,6 +705,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -829,6 +837,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(10),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -994,6 +1003,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1072,6 +1082,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1136,6 +1147,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1269,6 +1281,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1359,6 +1372,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				"dummy-path",
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1433,6 +1447,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1585,6 +1600,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			counter,
 			new StandaloneCompletedCheckpointStore(10),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1686,6 +1702,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1760,6 +1777,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1837,6 +1855,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1890,6 +1909,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			checkpointIDCounter,
 			new StandaloneCompletedCheckpointStore(2),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -1944,6 +1964,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(2),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -2007,6 +2028,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			store,
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -2122,6 +2144,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -2269,6 +2292,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -2553,6 +2577,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			standaloneCompletedCheckpointStore,
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -2701,6 +2726,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				"fake-directory",
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -3177,6 +3203,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -3356,6 +3383,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -3395,6 +3423,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			store,
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -3452,6 +3481,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			checkpointIDCounter,
 			completedCheckpointStore,
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -3545,6 +3575,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			new StandaloneCheckpointIDCounter(),
 			store,
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 				deleteExecutor -> {
 					SharedStateRegistry instance = new SharedStateRegistry(deleteExecutor);

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
index 47daa01..df2d37a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.SerializableObject;
 
 import org.hamcrest.BaseMatcher;
@@ -107,6 +108,7 @@ public class CheckpointStateRestoreTest {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -184,6 +186,7 @@ public class CheckpointStateRestoreTest {
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
 				null,
+				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
 
@@ -242,6 +245,7 @@ public class CheckpointStateRestoreTest {
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
 			null,
+			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 8f2531b..56a7e29 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -30,7 +30,6 @@ import org.apache.flink.api.common.io.FileInputFormat;
 import org.apache.flink.api.common.io.FilePathFilter;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -51,6 +50,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -135,7 +135,7 @@ public abstract class StreamExecutionEnvironment {
 	protected boolean isChainingEnabled = true;
 
 	/** The state backend used for storing k/v state and state snapshots. */
-	private AbstractStateBackend defaultStateBackend;
+	private StateBackend defaultStateBackend;
 
 	/** The time characteristic used by the data streams. */
 	private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;
@@ -431,12 +431,14 @@ public abstract class StreamExecutionEnvironment {
 	}
 
 	/**
-	 * Sets the state backend that describes how to store and checkpoint operator state. It defines in
-	 * what form the key/value state ({@link ValueState}, accessible
-	 * from operations on {@link org.apache.flink.streaming.api.datastream.KeyedStream}) is maintained
-	 * (heap, managed memory, externally), and where state snapshots/checkpoints are stored, both for
-	 * the key/value state, and for checkpointed functions (implementing the interface
-	 * {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction}).
+	 * Sets the state backend that describes how to store and checkpoint operator state. It defines
+	 * both which data structures hold state during execution (for example hash tables, RockDB,
+	 * or other data stores) as well as where checkpointed data will be persisted.
+	 *
+	 * <p>State managed by the state backend includes both keyed state that is accessible on
+	 * {@link org.apache.flink.streaming.api.datastream.KeyedStream keyed streams}, as well as
+	 * state maintained directly by the user code that implements
+	 * {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction CheckpointedFunction}.
 	 *
 	 * <p>The {@link org.apache.flink.runtime.state.memory.MemoryStateBackend} for example
 	 * maintains the state in heap memory, as objects. It is lightweight without extra dependencies,
@@ -453,19 +455,28 @@ public abstract class StreamExecutionEnvironment {
 	 * @see #getStateBackend()
 	 */
 	@PublicEvolving
+	public StreamExecutionEnvironment setStateBackend(StateBackend backend) {
+		this.defaultStateBackend = Preconditions.checkNotNull(backend);
+		return this;
+	}
+
+	/**
+	 * @deprecated Use {@link #setStateBackend(StateBackend)} instead.
+	 */
+	@Deprecated
+	@PublicEvolving
 	public StreamExecutionEnvironment setStateBackend(AbstractStateBackend backend) {
 		this.defaultStateBackend = Preconditions.checkNotNull(backend);
 		return this;
 	}
 
 	/**
-	 * Returns the state backend that defines how to store and checkpoint state.
-	 * @return The state backend that defines how to store and checkpoint state.
+	 * Gets the state backend that defines how to store and checkpoint state.
 	 *
-	 * @see #setStateBackend(AbstractStateBackend)
+	 * @see #setStateBackend(StateBackend)
 	 */
 	@PublicEvolving
-	public AbstractStateBackend getStateBackend() {
+	public StateBackend getStateBackend() {
 		return defaultStateBackend;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/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 20a361e..e5ed0c2 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
@@ -30,7 +30,7 @@ import org.apache.flink.api.java.typeutils.MissingTypeInfo;
 import org.apache.flink.optimizer.plan.StreamingPlan;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.environment.CheckpointConfig;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -92,7 +92,7 @@ public class StreamGraph extends StreamingPlan {
 
 	protected Map<Integer, String> vertexIDtoBrokerID;
 	protected Map<Integer, Long> vertexIDtoLoopTimeout;
-	private AbstractStateBackend stateBackend;
+	private StateBackend stateBackend;
 	private Set<Tuple2<StreamNode, StreamNode>> iterationSourceSinkPairs;
 
 	public StreamGraph(StreamExecutionEnvironment environment) {
@@ -143,11 +143,11 @@ public class StreamGraph extends StreamingPlan {
 		this.chaining = chaining;
 	}
 
-	public void setStateBackend(AbstractStateBackend backend) {
+	public void setStateBackend(StateBackend backend) {
 		this.stateBackend = backend;
 	}
 
-	public AbstractStateBackend getStateBackend() {
+	public StateBackend getStateBackend() {
 		return this.stateBackend;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d820d6f/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 3bba505..cd96dbf 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -27,6 +27,7 @@ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
 import org.apache.flink.api.scala.ClosureCleaner
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.state.AbstractStateBackend
+import org.apache.flink.runtime.state.StateBackend
 import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaEnv}
 import org.apache.flink.streaming.api.functions.source._
 import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
@@ -224,35 +225,45 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
   def getCheckpointingMode = javaEnv.getCheckpointingMode()
 
   /**
-   * Sets the state backend that describes how to store and checkpoint operator state.
-   * It defines in what form the key/value state, accessible from operations on
-   * [[KeyedStream]] is maintained (heap, managed memory, externally), and where state
-   * snapshots/checkpoints are stored, both for the key/value state, and for checkpointed
-   * functions (implementing the interface 
-   * [[org.apache.flink.streaming.api.checkpoint.CheckpointedFunction]].
+   * Sets the state backend that describes how to store and checkpoint operator state. It defines
+   * both which data structures hold state during execution (for example hash tables, RockDB,
+   * or other data stores) as well as where checkpointed data will be persisted.
    *
-   * <p>The [[org.apache.flink.runtime.state.memory.MemoryStateBackend]] for example
-   * maintains the state in heap memory, as objects. It is lightweight without extra 
-   * dependencies, but can checkpoint only small states (some counters).
+   * State managed by the state backend includes both keyed state that is accessible on
+   * [[org.apache.flink.streaming.api.datastream.KeyedStream keyed streams]], as well as
+   * state maintained directly by the user code that implements
+   * [[org.apache.flink.streaming.api.checkpoint.CheckpointedFunction CheckpointedFunction]].
    *
-   * <p>In contrast, the [[org.apache.flink.runtime.state.filesystem.FsStateBackend]]
-   * stores checkpoints of the state (also maintained as heap objects) in files. When using
-   * a replicated file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee
-   * that state is not lost upon failures of individual nodes and that the entire streaming
-   * program can be executed highly available and strongly consistent (assuming that Flink
-   * is run in high-availability mode).
+   * The [[org.apache.flink.runtime.state.memory.MemoryStateBackend]], for example,
+   * maintains the state in heap memory, as objects. It is lightweight without extra dependencies,
+   * but can checkpoint only small states (some counters).
+   *
+   * In contrast, the [[org.apache.flink.runtime.state.filesystem.FsStateBackend]]
+   * stores checkpoints of the state (also maintained as heap objects) in files.
+   * When using a replicated file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee
+   * that state is not lost upon failures of individual nodes and that streaming program can be
+   * executed highly available and strongly consistent.
    */
   @PublicEvolving
-  def setStateBackend(backend: AbstractStateBackend): StreamExecutionEnvironment = {
+  def setStateBackend(backend: StateBackend): StreamExecutionEnvironment = {
     javaEnv.setStateBackend(backend)
     this
   }
 
   /**
+   * @deprecated Use [[StreamExecutionEnvironment.setStateBackend(StateBackend)]] instead.
+   */
+  @Deprecated
+  @PublicEvolving
+  def setStateBackend(backend: AbstractStateBackend): StreamExecutionEnvironment = {
+    setStateBackend(backend.asInstanceOf[StateBackend])
+  }
+
+  /**
    * Returns the state backend that defines how to store and checkpoint state.
    */
   @PublicEvolving
-  def getStateBackend: AbstractStateBackend = javaEnv.getStateBackend()
+  def getStateBackend: StateBackend = javaEnv.getStateBackend()
 
   /**
     * Sets the restart strategy configuration. The configuration specifies which restart strategy


[04/17] flink git commit: [FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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
deleted file mode 100644
index 586df57..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.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;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Utilities for storing and loading savepoint meta data files.
- *
- * <p>Stored savepoints have the following format:
- * <pre>
- * MagicNumber SavepointVersion Savepoint
- *   - MagicNumber => int
- *   - SavepointVersion => int (returned by Savepoint#getVersion())
- *   - Savepoint => bytes (serialized via version-specific SavepointSerializer)
- * </pre>
- */
-public class SavepointStore {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SavepointStore.class);
-
-	/** Magic number for sanity checks against stored savepoints. */
-	public static final int MAGIC_NUMBER = 0x4960672d;
-
-	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.
-	 *
-	 * @param baseDirectory Base target directory for the savepoint
-	 * @param jobId Optional JobID the savepoint belongs to
-	 * @return The created savepoint directory
-	 * @throws IOException FileSystem operation failures are forwarded
-	 */
-	public static String createSavepointDirectory(@Nonnull String baseDirectory, @Nullable JobID jobId) throws IOException {
-		final Path basePath = new Path(baseDirectory);
-		final FileSystem fs = basePath.getFileSystem();
-
-		final String prefix;
-		if (jobId == null) {
-			prefix = "savepoint-";
-		} else {
-			prefix = String.format("savepoint-%s-", jobId.toString().substring(0, 6));
-		}
-
-		Exception latestException = null;
-
-		// Try to create a FS output stream
-		for (int attempt = 0; attempt < 10; attempt++) {
-			Path path = new Path(basePath, FileUtils.getRandomFilename(prefix));
-
-			try {
-				if (fs.mkdirs(path)) {
-					return path.toString();
-				}
-			} catch (Exception e) {
-				latestException = e;
-			}
-		}
-
-		throw new IOException("Failed to create savepoint directory at " + baseDirectory, latestException);
-	}
-
-	/**
-	 * Deletes a savepoint directory.
-	 *
-	 * @param savepointDirectory Recursively deletes the given directory
-	 * @throws IOException FileSystem operation failures are forwarded
-	 */
-	public static void deleteSavepointDirectory(@Nonnull String savepointDirectory) throws IOException {
-		Path path = new Path(savepointDirectory);
-		FileSystem fs = FileSystem.get(path.toUri());
-		fs.delete(path, true);
-	}
-
-	/**
-	 * Stores the savepoint metadata file.
-	 *
-	 * @param <T>       Savepoint type
-	 * @param directory Target directory to store savepoint in
-	 * @param savepoint Savepoint to be stored
-	 * @return Path of stored savepoint
-	 * @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_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
-		// 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 {
-		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
-	 */
-	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, filename);
-
-		final FileSystem fs = FileSystem.get(basePath.toUri());
-
-		boolean success = false;
-		try (FSDataOutputStream fdos = fs.create(metadataFilePath, WriteMode.NO_OVERWRITE);
-				DataOutputStream dos = new DataOutputStream(fdos))
-		{
-			// Write header
-			dos.writeInt(MAGIC_NUMBER);
-			dos.writeInt(savepoint.getVersion());
-
-			// 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)) {
-				if (!fs.delete(metadataFilePath, true)) {
-					LOG.warn("Failed to delete file {} after failed metadata write.", metadataFilePath);
-				}
-			}
-		}
-	}
-
-	/**
-	 * 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 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);
-
-		LOG.info("Loading savepoint from {}", path);
-
-		FileSystem fs = FileSystem.get(path.toUri());
-
-		FileStatus status = fs.getFileStatus(path);
-
-		// If this is a directory, we need to find the meta data file
-		if (status.isDir()) {
-			Path candidatePath = new Path(path, SAVEPOINT_METADATA_FILE);
-			if (fs.exists(candidatePath)) {
-				path = candidatePath;
-				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.");
-			}
-		}
-
-		// load the savepoint
-		final Savepoint savepoint;
-		try (DataInputStream dis = new DataInputViewStreamWrapper(fs.open(path))) {
-			int magicNumber = dis.readInt();
-
-			if (magicNumber == MAGIC_NUMBER) {
-				int version = dis.readInt();
-
-				SavepointSerializer<?> serializer = SavepointSerializers.getSerializer(version);
-				savepoint = serializer.deserialize(dis, classLoader);
-			} else {
-				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);
-	}
-
-	/**
-	 * Removes the savepoint meta data w/o loading and disposing it.
-	 *
-	 * @param path Path of savepoint to remove
-	 * @throws IOException Failures during disposal are forwarded
-	 */
-	public static void removeSavepointFile(String path) throws IOException {
-		Preconditions.checkNotNull(path, "Path");
-
-		try {
-			LOG.info("Removing savepoint: {}.", path);
-
-			Path filePath = new Path(path);
-			FileSystem fs = FileSystem.get(filePath.toUri());
-
-			if (fs.exists(filePath)) {
-				if (!fs.delete(filePath, true)) {
-					throw new IOException("Failed to delete " + filePath + ".");
-				}
-			} else {
-				throw new IllegalArgumentException("Invalid path '" + filePath.toUri() + "'.");
-			}
-		} catch (Throwable t) {
-			throw new IOException("Failed to dispose savepoint " + path + ".", t);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 f88a6b62..5c2f0f6 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
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.blob.PermanentBlobKey;
 import org.apache.flink.runtime.blob.VoidBlobWriter;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
 import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
@@ -58,7 +59,6 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
@@ -444,14 +444,13 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			long checkpointTimeout,
 			long minPauseBetweenCheckpoints,
 			int maxConcurrentCheckpoints,
-			ExternalizedCheckpointSettings externalizeSettings,
+			CheckpointRetentionPolicy retentionPolicy,
 			List<ExecutionJobVertex> verticesToTrigger,
 			List<ExecutionJobVertex> verticesToWaitFor,
 			List<ExecutionJobVertex> verticesToCommitTo,
 			List<MasterTriggerRestoreHook<?>> masterHooks,
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore checkpointStore,
-			String checkpointDir,
 			StateBackend checkpointStateBackend,
 			CheckpointStatsTracker statsTracker) {
 
@@ -475,13 +474,12 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			checkpointTimeout,
 			minPauseBetweenCheckpoints,
 			maxConcurrentCheckpoints,
-			externalizeSettings,
+			retentionPolicy,
 			tasksToTrigger,
 			tasksToWaitFor,
 			tasksToCommitTo,
 			checkpointIDCounter,
 			checkpointStore,
-			checkpointDir,
 			checkpointStateBackend,
 			ioExecutor,
 			SharedStateRegistry.DEFAULT_FACTORY);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 c742903..fe70bb6 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
@@ -186,7 +186,7 @@ public class ExecutionGraphBuilder {
 		// configure the state checkpointing
 		JobCheckpointingSettings snapshotSettings = jobGraph.getCheckpointingSettings();
 		if (snapshotSettings != null) {
-			List<ExecutionJobVertex> triggerVertices = 
+			List<ExecutionJobVertex> triggerVertices =
 					idToVertex(snapshotSettings.getVerticesToTrigger(), executionGraph);
 
 			List<ExecutionJobVertex> ackVertices =
@@ -242,7 +242,7 @@ public class ExecutionGraphBuilder {
 				try {
 					applicationConfiguredBackend = serializedAppConfigured.deserializeValue(classLoader);
 				} catch (IOException | ClassNotFoundException e) {
-					throw new JobExecutionException(jobId, 
+					throw new JobExecutionException(jobId,
 							"Could not deserialize application-defined state backend.", e);
 				}
 			}
@@ -295,14 +295,13 @@ public class ExecutionGraphBuilder {
 				chkConfig.getCheckpointTimeout(),
 				chkConfig.getMinPauseBetweenCheckpoints(),
 				chkConfig.getMaxConcurrentCheckpoints(),
-				chkConfig.getExternalizedCheckpointSettings(),
+				chkConfig.getCheckpointRetentionPolicy(),
 				triggerVertices,
 				ackVertices,
 				confirmVertices,
 				hooks,
 				checkpointIdCounter,
 				completedCheckpoints,
-				externalizedCheckpointsDir,
 				rootBackend,
 				checkpointStatsTracker);
 		}
@@ -331,7 +330,7 @@ public class ExecutionGraphBuilder {
 			} else {
 				throw new IllegalArgumentException(
 						"The snapshot checkpointing settings refer to non-existent vertex " + id);
-			} 
+			}
 		}
 
 		return result;
@@ -339,6 +338,6 @@ public class ExecutionGraphBuilder {
 
 	// ------------------------------------------------------------------------
 
-	/** This class is not supposed to be instantiated */
+	/** This class is not supposed to be instantiated. */
 	private ExecutionGraphBuilder() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java
index e00a6d4..4ecbda5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
@@ -31,7 +32,7 @@ import java.util.Objects;
  */
 public class CheckpointCoordinatorConfiguration implements Serializable {
 
-	private static final long serialVersionUID = -647384516034982626L;
+	private static final long serialVersionUID = 2L;
 
 	private final long checkpointInterval;
 
@@ -41,8 +42,8 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 
 	private final int maxConcurrentCheckpoints;
 
-	/** Settings for externalized checkpoints. */
-	private final ExternalizedCheckpointSettings externalizedCheckpointSettings;
+	/** Settings for what to do with checkpoints when a job finishes. */
+	private final CheckpointRetentionPolicy checkpointRetentionPolicy;
 
 	/**
 	 * Flag indicating whether exactly once checkpoint mode has been configured.
@@ -58,7 +59,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 			long checkpointTimeout,
 			long minPauseBetweenCheckpoints,
 			int maxConcurrentCheckpoints,
-			ExternalizedCheckpointSettings externalizedCheckpointSettings,
+			CheckpointRetentionPolicy checkpointRetentionPolicy,
 			boolean isExactlyOnce) {
 
 		// sanity checks
@@ -71,7 +72,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 		this.checkpointTimeout = checkpointTimeout;
 		this.minPauseBetweenCheckpoints = minPauseBetweenCheckpoints;
 		this.maxConcurrentCheckpoints = maxConcurrentCheckpoints;
-		this.externalizedCheckpointSettings = Preconditions.checkNotNull(externalizedCheckpointSettings);
+		this.checkpointRetentionPolicy = Preconditions.checkNotNull(checkpointRetentionPolicy);
 		this.isExactlyOnce = isExactlyOnce;
 	}
 
@@ -91,8 +92,8 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 		return maxConcurrentCheckpoints;
 	}
 
-	public ExternalizedCheckpointSettings getExternalizedCheckpointSettings() {
-		return externalizedCheckpointSettings;
+	public CheckpointRetentionPolicy getCheckpointRetentionPolicy() {
+		return checkpointRetentionPolicy;
 	}
 
 	public boolean isExactlyOnce() {
@@ -113,12 +114,18 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 			minPauseBetweenCheckpoints == that.minPauseBetweenCheckpoints &&
 			maxConcurrentCheckpoints == that.maxConcurrentCheckpoints &&
 			isExactlyOnce == that.isExactlyOnce &&
-			Objects.equals(externalizedCheckpointSettings, that.externalizedCheckpointSettings);
+			checkpointRetentionPolicy == that.checkpointRetentionPolicy;
 	}
 
 	@Override
 	public int hashCode() {
-		return Objects.hash(checkpointInterval, checkpointTimeout, minPauseBetweenCheckpoints, maxConcurrentCheckpoints, externalizedCheckpointSettings, isExactlyOnce);
+		return Objects.hash(
+				checkpointInterval,
+				checkpointTimeout,
+				minPauseBetweenCheckpoints,
+				maxConcurrentCheckpoints,
+				checkpointRetentionPolicy,
+				isExactlyOnce);
 	}
 
 	@Override
@@ -128,7 +135,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable {
 			", checkpointTimeout=" + checkpointTimeout +
 			", minPauseBetweenCheckpoints=" + minPauseBetweenCheckpoints +
 			", maxConcurrentCheckpoints=" + maxConcurrentCheckpoints +
-			", externalizedCheckpointSettings=" + externalizedCheckpointSettings +
+			", checkpointRetentionPolicy=" + checkpointRetentionPolicy +
 			'}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/ExternalizedCheckpointSettings.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/ExternalizedCheckpointSettings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/ExternalizedCheckpointSettings.java
deleted file mode 100644
index f432796..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/ExternalizedCheckpointSettings.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.jobgraph.tasks;
-
-import org.apache.flink.annotation.Internal;
-
-import java.util.Objects;
-
-/**
- * Grouped settings for externalized checkpoints.
- */
-@Internal
-public class ExternalizedCheckpointSettings implements java.io.Serializable {
-
-	private static final long serialVersionUID = -6271691851124392955L;
-
-	private static final ExternalizedCheckpointSettings NONE = new ExternalizedCheckpointSettings(false, false);
-
-	/** Flag indicating whether checkpoints should be externalized. */
-	private final boolean externalizeCheckpoints;
-
-	/** Flag indicating whether externalized checkpoints should delete on cancellation. */
-	private final boolean deleteOnCancellation;
-
-	private ExternalizedCheckpointSettings(boolean externalizeCheckpoints, boolean deleteOnCancellation) {
-		this.externalizeCheckpoints = externalizeCheckpoints;
-		this.deleteOnCancellation = deleteOnCancellation;
-	}
-
-	/**
-	 * Returns <code>true</code> if checkpoints should be externalized.
-	 *
-	 * @return <code>true</code> if checkpoints should be externalized.
-	 */
-	public boolean externalizeCheckpoints() {
-		return externalizeCheckpoints;
-	}
-
-	/**
-	 * Returns <code>true</code> if externalized checkpoints should be deleted on cancellation.
-	 *
-	 * @return <code>true</code> if externalized checkpoints should be deleted on cancellation.
-	 */
-	public boolean deleteOnCancellation() {
-		return deleteOnCancellation;
-	}
-
-	public static ExternalizedCheckpointSettings externalizeCheckpoints(boolean deleteOnCancellation) {
-		return new ExternalizedCheckpointSettings(true, deleteOnCancellation);
-	}
-
-	public static ExternalizedCheckpointSettings none() {
-		return NONE;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-		ExternalizedCheckpointSettings that = (ExternalizedCheckpointSettings) o;
-		return externalizeCheckpoints == that.externalizeCheckpoints &&
-			deleteOnCancellation == that.deleteOnCancellation;
-	}
-
-	@Override
-	public int hashCode() {
-		return Objects.hash(externalizeCheckpoints, deleteOnCancellation);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java
index 91c9ae1..b88183e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.rest.handler.job.checkpoints;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.rest.handler.HandlerRequest;
 import org.apache.flink.runtime.rest.handler.RestHandlerException;
 import org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler;
@@ -71,11 +71,11 @@ public class CheckpointConfigHandler extends AbstractExecutionGraphHandler<Check
 				"Checkpointing is not enabled for this job (" + executionGraph.getJobID() + ").",
 				HttpResponseStatus.NOT_FOUND);
 		} else {
-			ExternalizedCheckpointSettings externalizedCheckpointSettings = checkpointCoordinatorConfiguration.getExternalizedCheckpointSettings();
+			CheckpointRetentionPolicy retentionPolicy = checkpointCoordinatorConfiguration.getCheckpointRetentionPolicy();
 
 			CheckpointConfigInfo.ExternalizedCheckpointInfo externalizedCheckpointInfo = new CheckpointConfigInfo.ExternalizedCheckpointInfo(
-				externalizedCheckpointSettings.externalizeCheckpoints(),
-				externalizedCheckpointSettings.deleteOnCancellation());
+					retentionPolicy != CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
+					retentionPolicy != CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION);
 
 			return new CheckpointConfigInfo(
 				checkpointCoordinatorConfiguration.isExactlyOnce() ? CheckpointConfigInfo.ProcessingMode.EXACTLY_ONCE : CheckpointConfigInfo.ProcessingMode.AT_LEAST_ONCE,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
index 69cc55f..fbaa484 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.checkpoints;
 
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.rest.handler.legacy.AbstractExecutionGraphRequestHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
 import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
@@ -100,12 +100,13 @@ public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandle
 			gen.writeNumberField(CheckpointConfigInfo.FIELD_NAME_CHECKPOINT_MIN_PAUSE, jobCheckpointingConfiguration.getMinPauseBetweenCheckpoints());
 			gen.writeNumberField(CheckpointConfigInfo.FIELD_NAME_CHECKPOINT_MAX_CONCURRENT, jobCheckpointingConfiguration.getMaxConcurrentCheckpoints());
 
-			ExternalizedCheckpointSettings externalization = jobCheckpointingConfiguration.getExternalizedCheckpointSettings();
+			CheckpointRetentionPolicy retentionPolicy = jobCheckpointingConfiguration.getCheckpointRetentionPolicy();
 			gen.writeObjectFieldStart(CheckpointConfigInfo.FIELD_NAME_EXTERNALIZED_CHECKPOINT_CONFIG);
 			{
-				if (externalization.externalizeCheckpoints()) {
+				if (retentionPolicy != CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION) {
 					gen.writeBooleanField(CheckpointConfigInfo.ExternalizedCheckpointInfo.FIELD_NAME_ENABLED, true);
-					gen.writeBooleanField(CheckpointConfigInfo.ExternalizedCheckpointInfo.FIELD_NAME_DELETE_ON_CANCELLATION, externalization.deleteOnCancellation());
+					gen.writeBooleanField(CheckpointConfigInfo.ExternalizedCheckpointInfo.FIELD_NAME_DELETE_ON_CANCELLATION,
+							retentionPolicy != CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION);
 				} else {
 					gen.writeBooleanField(CheckpointConfigInfo.ExternalizedCheckpointInfo.FIELD_NAME_ENABLED, false);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
new file mode 100644
index 0000000..ea1015a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
@@ -0,0 +1,93 @@
+/*
+ * 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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/**
+ * CheckpointStorage implements the durable storage of checkpoint data and metadata streams.
+ * An individual checkpoint or savepoint is stored to a {@link CheckpointStorageLocation},
+ * created by this class.
+ */
+public interface CheckpointStorage {
+
+	/**
+	 * Checks whether this backend supports highly available storage of data.
+	 *
+	 * <p>Some state backends may offer support for that with default settings, which makes them
+	 * suitable for zero-config prototyping, but not for actual production setups.
+	 */
+	boolean supportsHighlyAvailableStorage();
+
+	/**
+	 * Checks whether the storage has a default savepoint location configured.
+	 */
+	boolean hasDefaultSavepointLocation();
+
+	/**
+	 * Resolves the given pointer to a checkpoint/savepoint into a state handle from which the
+	 * checkpoint metadata can be read. If the state backend cannot understand the format of
+	 * the pointer (for example because it was created by a different state backend) this method
+	 * should throw an {@code IOException}.
+	 *
+	 * @param pointer The pointer to resolve.
+	 * @return The state handler from which one can read the checkpoint metadata.
+	 *
+	 * @throws IOException Thrown, if the state backend does not understand the pointer, or if
+	 *                     the pointer could not be resolved due to an I/O error.
+	 */
+	StreamStateHandle resolveCheckpoint(String pointer) throws IOException;
+
+	/**
+	 * Initializes a storage location for new checkpoint with the given ID.
+	 *
+	 * <p>The returned storage location can be used to write the checkpoint data and metadata
+	 * to and to obtain the pointers for the location(s) where the actual checkpoint data should be
+	 * stored.
+	 *
+	 * @param checkpointId The ID (logical timestamp) of the checkpoint that should be persisted.
+	 * @return A storage location for the data and metadata of the given checkpoint.
+	 *
+	 * @throws IOException Thrown if the storage location cannot be initialized due to an I/O exception.
+	 */
+	CheckpointStorageLocation initializeLocationForCheckpoint(long checkpointId) throws IOException;
+
+	/**
+	 * Initializes a storage location for new savepoint with the given ID.
+	 *
+	 * <p>If an external location pointer is passed, the savepoint storage location
+	 * will be initialized at the location of that pointer. If the external location pointer is null,
+	 * the default savepoint location will be used. If no default savepoint location is configured,
+	 * this will throw an exception. Whether a default savepoint location is configured can be
+	 * checked via {@link #hasDefaultSavepointLocation()}.
+	 *
+	 * @param checkpointId The ID (logical timestamp) of the savepoint's checkpoint.
+	 * @param externalLocationPointer Optionally, a pointer to the location where the savepoint should
+	 *                                be stored. May be null.
+	 *
+	 * @return A storage location for the data and metadata of the savepoint.
+	 *
+	 * @throws IOException Thrown if the storage location cannot be initialized due to an I/O exception.
+	 */
+	CheckpointStorageLocation initializeLocationForSavepoint(
+			long checkpointId,
+			@Nullable String externalLocationPointer) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageLocation.java
new file mode 100644
index 0000000..fbc4805
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageLocation.java
@@ -0,0 +1,65 @@
+/*
+ * 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.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+
+import java.io.IOException;
+
+/**
+ * A storage location for one particular checkpoint. This location is typically
+ * created and initialized via {@link CheckpointStorage#initializeLocationForCheckpoint(long)} or
+ * {@link CheckpointStorage#initializeLocationForSavepoint(long, String)}.
+ */
+public interface CheckpointStorageLocation {
+
+	/**
+	 * Creates the output stream to persist the checkpoint metadata to.
+	 *
+	 * @return The output stream to persist the checkpoint metadata to.
+	 * @throws IOException Thrown, if the stream cannot be opened due to an I/O error.
+	 */
+	CheckpointStateOutputStream createMetadataOutputStream() throws IOException;
+
+	/**
+	 * Finalizes the checkpoint, marking the location as a finished checkpoint.
+	 * This method returns the external checkpoint pointer that can be used to resolve
+	 * the checkpoint upon recovery.
+	 *
+	 * @return The external pointer to the checkpoint at this location.
+	 * @throws IOException Thrown, if finalizing / marking as finished fails due to an I/O error.
+	 */
+	String markCheckpointAsFinished() throws IOException;
+
+	/**
+	 * Disposes the checkpoint location in case the checkpoint has failed.
+	 */
+	void disposeOnFailure() throws IOException;
+
+	/**
+	 * Gets the location encoded as a string pointer.
+	 *
+	 * <p>This pointer is used to send the target storage location via checkpoint RPC messages
+	 * and checkpoint barriers, in a format avoiding backend-specific classes.
+	 *
+	 * <p>That string encodes the location typically in a backend-specific way.
+	 * For example, file-based backends can encode paths here.
+	 */
+	String getLocationAsPointer();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java
index 73113ec..c94b0e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.core.fs.FSDataOutputStream;
@@ -42,6 +43,14 @@ public interface CheckpointStreamFactory {
 	/**
 	 * A dedicated output stream that produces a {@link StreamStateHandle} when closed.
 	 *
+	 * <p><b>Important:</b> When closing this stream after the successful case, you must
+	 * call {@link #closeAndGetHandle()} - only that method will actually retain the resource
+	 * written to. The method has the semantics of "close on success".
+	 * The {@link #close()} method is supposed to remove the target resource if
+	 * called before {@link #closeAndGetHandle()}, hence having the semantics of
+	 * "close on failure". That way, simple try-with-resources statements automatically
+	 * clean up unsuccessful partial state resources in case the writing does not complete.
+	 *
 	 * <p>Note: This is an abstract class and not an interface because {@link OutputStream}
 	 * is an abstract class.
 	 */
@@ -51,9 +60,27 @@ public interface CheckpointStreamFactory {
 		 * Closes the stream and gets a state handle that can create an input stream
 		 * producing the data written to this stream.
 		 *
+		 * <p>This closing must be called (also when the caller is not interested in the handle)
+		 * to successfully close the stream and retain the produced resource. In contrast,
+		 * the {@link #close()} method removes the target resource when called.
+		 *
 		 * @return A state handle that can create an input stream producing the data written to this stream.
 		 * @throws IOException Thrown, if the stream cannot be closed.
 		 */
 		public abstract StreamStateHandle closeAndGetHandle() throws IOException;
+
+		/**
+		 * This method should close the stream, if has not been closed before.
+		 * If this method actually closes the stream, it should delete/release the
+		 * resource behind the stream, such as the file that the stream writes to.
+		 *
+		 * <p>The above implies that this method is intended to be the "unsuccessful close",
+		 * such as when cancelling the stream writing, or when an exception occurs.
+		 * Closing the stream for the successful case must go through {@link #closeAndGetHandle()}.
+		 *
+		 * @throws IOException Thrown, if the stream cannot be closed.
+		 */
+		@Override
+		public abstract void close() throws IOException;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
index 7961b5e..3d3fda2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 
 import javax.annotation.Nullable;
+
 import java.io.IOException;
 
 /**
@@ -73,8 +74,7 @@ import java.io.IOException;
  * states stores that provide access to the persistent storage and hold the keyed- and operator
  * state data structures. That way, the State Backend can be very lightweight (contain only
  * configurations) which makes it easier to be serializable.
- * 
- * 
+ *
  * <h2>Thread Safety</h2>
  * 
  * State backend implementations have to be thread-safe. Multiple threads may be creating
@@ -84,13 +84,44 @@ import java.io.IOException;
 public interface StateBackend extends java.io.Serializable {
 
 	// ------------------------------------------------------------------------
-	//  Persistent Bytes Storage
+	//  Checkpoint storage - the durable persistence of checkpoint data
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Resolves the given pointer to a checkpoint/savepoint into a state handle from which the
+	 * checkpoint metadata can be read. If the state backend cannot understand the format of
+	 * the pointer (for example because it was created by a different state backend) this method
+	 * should throw an {@code IOException}.
+	 *
+	 * @param pointer The pointer to resolve.
+	 * @return The state handler from which one can read the checkpoint metadata.
+	 *
+	 * @throws IOException Thrown, if the state backend does not understand the pointer, or if
+	 *                     the pointer could not be resolved due to an I/O error.
+	 */
+	StreamStateHandle resolveCheckpoint(String pointer) throws IOException;
+
+	/**
+	 * Creates a storage for checkpoints for the given job. The checkpoint storage is
+	 * used to write checkpoint data and metadata.
+	 *
+	 * @param jobId The job to store checkpoint data for.
+	 * @return A checkpoint storage for the given job.
+	 *
+	 * @throws IOException Thrown if the checkpoint storage cannot be initialized.
+	 */
+	CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException;
+
+	// ------------------------------------------------------------------------
+	//  Persistent bytes storage for checkpoint data
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Creates a {@link CheckpointStreamFactory} that can be used to create streams
 	 * that should end up in a checkpoint.
 	 *
+	 * <p>NOTE: This method will probably go into the {@link CheckpointStorage} in the future.
+	 *
 	 * @param jobId              The {@link JobID} of the job for which we are creating checkpoint streams.
 	 * @param operatorIdentifier An identifier of the operator for which we create streams.
 	 */
@@ -104,6 +135,8 @@ public interface StateBackend extends java.io.Serializable {
 	 * this will return the same factory as for regular checkpoints, but maybe
 	 * slightly adjusted.
 	 *
+	 * <p>NOTE: This method will probably go into the {@link CheckpointStorage} in the future.
+	 *
 	 * @param jobId The {@link JobID} of the job for which we are creating checkpoint streams.
 	 * @param operatorIdentifier An identifier of the operator for which we create streams.
 	 * @param targetLocation An optional custom location for the savepoint stream.
@@ -124,21 +157,13 @@ public interface StateBackend extends java.io.Serializable {
 	/**
 	 * Creates a new {@link AbstractKeyedStateBackend} that is responsible for holding <b>keyed state</b>
 	 * and checkpointing it.
-	 * 
+	 *
 	 * <p><i>Keyed State</i> is state where each value is bound to a key.
-	 * 
-	 * @param env
-	 * @param jobID
-	 * @param operatorIdentifier
-	 * @param keySerializer
-	 * @param numberOfKeyGroups
-	 * @param keyGroupRange
-	 * @param kvStateRegistry
-	 * 
+	 *
 	 * @param <K> The type of the keys by which the state is organized.
-	 *     
+	 *
 	 * @return The Keyed State Backend for the given job, operator, and key group range.
-	 * 
+	 *
 	 * @throws Exception This method may forward all exceptions that occur while instantiating the backend.
 	 */
 	<K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
@@ -149,18 +174,18 @@ public interface StateBackend extends java.io.Serializable {
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
 			TaskKvStateRegistry kvStateRegistry) throws Exception;
-
+	
 	/**
 	 * Creates a new {@link OperatorStateBackend} that can be used for storing operator state.
-	 * 
+	 *
 	 * <p>Operator state is state that is associated with parallel operator (or function) instances,
 	 * rather than with keys.
-	 * 
+	 *
 	 * @param env The runtime environment of the executing task.
 	 * @param operatorIdentifier The identifier of the operator whose state should be stored.
-	 * 
+	 *
 	 * @return The OperatorStateBackend for operator identified by the job and operator identifier.
-	 * 
+	 *
 	 * @throws Exception This method may forward all exceptions that occur while instantiating the backend.
 	 */
 	OperatorStateBackend createOperatorStateBackend(Environment env, String operatorIdentifier) throws Exception;

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
index 6ec6f24..0b622c9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
@@ -25,11 +25,11 @@ 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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.flink.runtime.state.StreamStateHandle;
 
 import javax.annotation.Nullable;
+
+import java.io.IOException;
 import java.net.URI;
 
 /**
@@ -45,8 +45,8 @@ import java.net.URI;
  *
  * <h1>Checkpoint Layout</h1>
  *
- * The state backend is configured with a base directory and persists the checkpoint data of specific
- * checkpoints in specific subdirectories. For example, if the base directory was set to 
+ * <p>The state backend is configured with a base directory and persists the checkpoint data of specific
+ * checkpoints in specific subdirectories. For example, if the base directory was set to
  * {@code hdfs://namenode:port/flink-checkpoints/}, the state backend will create a subdirectory with
  * the job's ID that will contain the actual checkpoints:
  * ({@code hdfs://namenode:port/flink-checkpoints/1b080b6e710aabbef8993ab18c6de98b})
@@ -56,17 +56,20 @@ import java.net.URI;
  *
  * <h1>Savepoint Layout</h1>
  *
- * A savepoint that is set to be stored in path {@code hdfs://namenode:port/flink-savepoints/}, will create
+ * <p>A savepoint that is set to be stored in path {@code hdfs://namenode:port/flink-savepoints/}, will create
  * a subdirectory {@code savepoint-jobId(0, 6)-randomDigits} in which it stores all savepoint data.
  * The random digits are added as "entropy" to avoid directory collisions.
+ *
+ * <h1>Metadata File</h1>
+ *
+ * <p>A completed checkpoint writes its metadata into a file
+ * '{@value AbstractFsCheckpointStorage#METADATA_FILE_NAME}'.
  */
 @PublicEvolving
 public abstract class AbstractFileStateBackend extends AbstractStateBackend {
 
 	private static final long serialVersionUID = 1L;
 
-	private static final Logger LOG = LoggerFactory.getLogger(AbstractFileStateBackend.class);
-
 	// ------------------------------------------------------------------------
 	//  State Backend Properties
 	// ------------------------------------------------------------------------
@@ -118,7 +121,7 @@ public abstract class AbstractFileStateBackend extends AbstractStateBackend {
 	 *
 	 * @param baseCheckpointPath The checkpoint base directory to use (or null).
 	 * @param baseSavepointPath The default savepoint directory to use (or null).
-	 * @param configuration The configuration to read values from 
+	 * @param configuration The configuration to read values from.
 	 */
 	protected AbstractFileStateBackend(
 			@Nullable Path baseCheckpointPath,
@@ -154,10 +157,18 @@ public abstract class AbstractFileStateBackend extends AbstractStateBackend {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Initialization and metadata storage
+	// ------------------------------------------------------------------------
+
+	@Override
+	public StreamStateHandle resolveCheckpoint(String pointer) throws IOException {
+		return AbstractFsCheckpointStorage.resolveCheckpointPointer(pointer);
+	}
+
+	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	// 
 	/**
 	 * Checks the validity of the path's scheme and path.
 	 *
@@ -203,4 +214,4 @@ public abstract class AbstractFileStateBackend extends AbstractStateBackend {
 			}
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsCheckpointStorage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsCheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsCheckpointStorage.java
new file mode 100644
index 0000000..1254ee2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsCheckpointStorage.java
@@ -0,0 +1,256 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.FileUtils;
+
+import javax.annotation.Nullable;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of durable checkpoint storage to file systems.
+ */
+public abstract class AbstractFsCheckpointStorage implements CheckpointStorage {
+
+	// ------------------------------------------------------------------------
+	//  Constants
+	// ------------------------------------------------------------------------
+
+	/** The prefix of the directory containing the data exclusive to a checkpoint. */
+	public static final String CHECKPOINT_DIR_PREFIX = "chk-";
+
+	/** The name of the directory for shared checkpoint state. */
+	public static final String CHECKPOINT_SHARED_STATE_DIR = "shared";
+
+	/** The name of the directory for state not owned/released by the master, but by the TaskManagers. */
+	public static final String CHECKPOINT_TASK_OWNED_STATE_DIR = "taskowned";
+
+	/** The name of the metadata files in checkpoints / savepoints. */
+	public static final String METADATA_FILE_NAME = "_metadata";
+
+	// ------------------------------------------------------------------------
+	//  Fields and properties
+	// ------------------------------------------------------------------------
+
+	/** The jobId, written into the generated savepoint directories. */
+	private final JobID jobId;
+
+	/** The default location for savepoints. Null, if none is configured. */
+	@Nullable
+	private final Path defaultSavepointDirectory;
+
+	/**
+	 * Creates a new checkpoint storage.
+	 *
+	 * @param jobId The ID of the job that writes the checkpoints.
+	 * @param defaultSavepointDirectory The default location for savepoints, or null, if none is set.
+	 */
+	protected AbstractFsCheckpointStorage(
+			JobID jobId,
+			@Nullable Path defaultSavepointDirectory) {
+
+		this.jobId = checkNotNull(jobId);
+		this.defaultSavepointDirectory = defaultSavepointDirectory;
+	}
+
+	/**
+	 * Gets the default directory for savepoints. Returns null, if no default savepoint
+	 * directory is configured.
+	 */
+	@Nullable
+	public Path getDefaultSavepointDirectory() {
+		return defaultSavepointDirectory;
+	}
+
+	// ------------------------------------------------------------------------
+	//  CheckpointStorage implementation
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean hasDefaultSavepointLocation() {
+		return defaultSavepointDirectory != null;
+	}
+
+	@Override
+	public StreamStateHandle resolveCheckpoint(String checkpointPointer) throws IOException {
+		return resolveCheckpointPointer(checkpointPointer);
+	}
+
+	/**
+	 * Creates a file system based storage location for a savepoint.
+	 *
+	 * <p>This methods implements the logic that decides which location to use (given optional
+	 * parameters for a configured location and a location passed for this specific savepoint)
+	 * and how to name and initialize the savepoint directory.
+	 *
+	 * @param externalLocationPointer    The target location pointer for the savepoint.
+	 *                                   Must be a valid URI. Null, if not supplied.
+	 * @param checkpointId               The checkpoint ID of the savepoint.
+	 *
+	 * @return The checkpoint storage location for the savepoint.
+	 *
+	 * @throws IOException Thrown if the target directory could not be created.
+	 */
+	@Override
+	public FsCheckpointStorageLocation initializeLocationForSavepoint(
+			@SuppressWarnings("unused") long checkpointId,
+			@Nullable String externalLocationPointer) throws IOException {
+
+		// determine where to write the savepoint to
+
+		final Path savepointBasePath;
+		if (externalLocationPointer != null) {
+			savepointBasePath = new Path(externalLocationPointer);
+		}
+		else if (defaultSavepointDirectory != null) {
+			savepointBasePath = defaultSavepointDirectory;
+		}
+		else {
+			throw new IllegalArgumentException("No savepoint location given and no default location configured.");
+		}
+
+		// generate the savepoint directory
+
+		final FileSystem fs = savepointBasePath.getFileSystem();
+		final String prefix = "savepoint-" + jobId.toString().substring(0, 6) + '-';
+
+		Exception latestException = null;
+		for (int attempt = 0; attempt < 10; attempt++) {
+			final Path path = new Path(savepointBasePath, FileUtils.getRandomFilename(prefix));
+
+			try {
+				if (fs.mkdirs(path)) {
+					return new FsCheckpointStorageLocation(fs, path, path, path);
+				}
+			} catch (Exception e) {
+				latestException = e;
+			}
+		}
+
+		throw new IOException("Failed to create savepoint directory at " + savepointBasePath, latestException);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Creating and resolving paths
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Builds directory into which a specific job checkpoints, meaning the directory inside which
+	 * it creates the checkpoint-specific subdirectories.
+	 *
+	 * <p>This method only succeeds if a base checkpoint directory has been set; otherwise
+	 * the method fails with an exception.
+	 *
+	 * @param jobId The ID of the job
+	 * @return The job's checkpoint directory, re
+	 *
+	 * @throws UnsupportedOperationException Thrown, if no base checkpoint directory has been set.
+	 */
+	protected static Path getCheckpointDirectoryForJob(Path baseCheckpointPath, JobID jobId) {
+		return new Path(baseCheckpointPath, jobId.toString());
+	}
+
+	/**
+	 * Creates the directory path for the data exclusive to a specific checkpoint.
+	 *
+	 * @param baseDirectory The base directory into which the job checkpoints.
+	 * @param checkpointId The ID (logical timestamp) of the checkpoint.
+	 */
+	protected static Path createCheckpointDirectory(Path baseDirectory, long checkpointId) {
+		return new Path(baseDirectory, CHECKPOINT_DIR_PREFIX + checkpointId);
+	}
+
+	/**
+	 * Takes the given string (representing a pointer to a checkpoint) and resolves it to a file
+	 * status for the checkpoint's metadata file.
+	 *
+	 * @param checkpointPointer The pointer to resolve.
+	 * @return A state handle to checkpoint/savepoint's metadata.
+	 *
+	 * @throws IOException Thrown, if the pointer cannot be resolved, the file system not accessed, or
+	 *                     the pointer points to a location that does not seem to be a checkpoint/savepoint.
+	 */
+	protected static StreamStateHandle resolveCheckpointPointer(String checkpointPointer) throws IOException {
+		checkNotNull(checkpointPointer, "checkpointPointer");
+		checkArgument(!checkpointPointer.isEmpty(), "empty checkpoint pointer");
+
+		// check if the pointer is in fact a valid file path
+		final Path path;
+		try {
+			path = new Path(checkpointPointer);
+		}
+		catch (Exception e) {
+			throw new IOException("Checkpoint/savepoint path '" + checkpointPointer + "' is not a valid file URI. " +
+					"Either the pointer path is invalid, or the checkpoint was created by a different state backend.");
+		}
+
+		// check if the file system can be accessed
+		final FileSystem fs;
+		try {
+			fs = path.getFileSystem();
+		}
+		catch (IOException e) {
+			throw new IOException("Cannot access file system for checkpoint/savepoint path '" +
+					checkpointPointer + "'.", e);
+		}
+
+		final FileStatus status;
+		try {
+			status = fs.getFileStatus(path);
+		}
+		catch (FileNotFoundException e) {
+			throw new FileNotFoundException("Cannot find checkpoint or savepoint " +
+					"file/directory '" + checkpointPointer + "' on file system '" + fs.getUri().getScheme() + "'.");
+		}
+
+		// if we are here, the file / directory exists
+		final FileStatus metadataFileStatus;
+
+		// If this is a directory, we need to find the meta data file
+		if (status.isDir()) {
+			final Path metadataFilePath = new Path(path, METADATA_FILE_NAME);
+			try {
+				metadataFileStatus = fs.getFileStatus(metadataFilePath);
+			}
+			catch (FileNotFoundException e) {
+				throw new FileNotFoundException("Cannot find meta data file '" + METADATA_FILE_NAME +
+						"' in directory '" + path + "'. Please try to load the checkpoint/savepoint " +
+						"directly from the metadata file instead of the directory.");
+			}
+		}
+		else {
+			// this points to a file and we either do no name validation, or
+			// the name is actually correct, so we can return the path
+			metadataFileStatus = status;
+		}
+
+		return new FileStateHandle(metadataFileStatus.getPath(), metadataFileStatus.getLen());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStream.java
new file mode 100644
index 0000000..bc1ca01
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FixFileFsStateOutputStream.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.filesystem;
+
+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 org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link CheckpointStateOutputStream} that writes into a specified file and
+ * returns a {@link FileStateHandle} upon closing.
+ *
+ * <p>Unlike the {@link org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.FsCheckpointStateOutputStream},
+ * this stream does not have a threshold below which it returns a memory byte stream handle,
+ * and does not create random files, but writes to a specified file. 
+ */
+public final class FixFileFsStateOutputStream extends CheckpointStateOutputStream {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FixFileFsStateOutputStream.class);
+
+	// ------------------------------------------------------------------------
+
+	private final FSDataOutputStream out;
+
+	private final Path path;
+
+	private final FileSystem fileSystem;
+
+	private volatile boolean closed;
+
+
+	public FixFileFsStateOutputStream(FileSystem fileSystem, Path path) throws IOException {
+		this.fileSystem = checkNotNull(fileSystem);
+		this.path = checkNotNull(path);
+
+		this.out = fileSystem.create(path, WriteMode.NO_OVERWRITE);
+	}
+
+	// ------------------------------------------------------------------------
+	//  I/O
+	// ------------------------------------------------------------------------
+
+	@Override
+	public final void write(int b) throws IOException {
+		out.write(b);
+	}
+
+	@Override
+	public final void write(@Nonnull byte[] b, int off, int len) throws IOException {
+		out.write(b, off, len);
+	}
+
+	@Override
+	public long getPos() throws IOException {
+		return out.getPos();
+	}
+
+	@Override
+	public void flush() throws IOException {
+		out.flush();
+	}
+
+	@Override
+	public void sync() throws IOException {
+		out.sync();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Closing
+	// ------------------------------------------------------------------------
+
+	public boolean isClosed() {
+		return closed;
+	}
+
+	@Override
+	public void close() {
+		if (!closed) {
+			closed = true;
+
+			try {
+				out.close();
+				fileSystem.delete(path, false);
+			}
+			catch (Throwable t) {
+				LOG.warn("Could not close the state stream for {}.", path, t);
+			}
+		}
+	}
+
+	@Override
+	public FileStateHandle closeAndGetHandle() throws IOException {
+		synchronized (this) {
+			if (!closed) {
+				try {
+					// make a best effort attempt to figure out the size
+					long size = 0;
+					try {
+						size = out.getPos();
+					} catch (Exception ignored) {}
+
+					// close and return
+					out.close();
+
+					return new FileStateHandle(path, size);
+				}
+				catch (Exception e) {
+					try {
+						fileSystem.delete(path, false);
+					}
+					catch (Exception deleteException) {
+						LOG.warn("Could not delete the checkpoint stream file {}.", path, deleteException);
+					}
+
+					throw new IOException("Could not flush and close the file system " +
+							"output stream to " + path + " in order to obtain the " +
+							"stream state handle", e);
+				}
+				finally {
+					closed = true;
+				}
+			}
+			else {
+				throw new IOException("Stream has already been closed and discarded.");
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
new file mode 100644
index 0000000..b7be8fa
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
@@ -0,0 +1,85 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * An implementation of durable checkpoint storage to file systems.
+ */
+public class FsCheckpointStorage extends AbstractFsCheckpointStorage {
+
+	private final FileSystem fileSystem;
+
+	private final Path checkpointsDirectory;
+
+	private final Path sharedStateDirectory;
+
+	private final Path taskOwnedStateDirectory;
+
+	public FsCheckpointStorage(
+			Path checkpointBaseDirectory,
+			@Nullable Path defaultSavepointDirectory,
+			JobID jobId) throws IOException {
+
+		super(jobId, defaultSavepointDirectory);
+
+		this.fileSystem = checkpointBaseDirectory.getFileSystem();
+		this.checkpointsDirectory = getCheckpointDirectoryForJob(checkpointBaseDirectory, jobId);
+		this.sharedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_SHARED_STATE_DIR);
+		this.taskOwnedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_TASK_OWNED_STATE_DIR);
+
+		// initialize the dedicated directories
+		fileSystem.mkdirs(checkpointsDirectory);
+		fileSystem.mkdirs(sharedStateDirectory);
+		fileSystem.mkdirs(taskOwnedStateDirectory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  CheckpointStorage implementation
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsHighlyAvailableStorage() {
+		return true;
+	}
+
+	@Override
+	public CheckpointStorageLocation initializeLocationForCheckpoint(long checkpointId) throws IOException {
+		checkArgument(checkpointId >= 0);
+
+		// prepare all the paths needed for the checkpoints
+		final Path checkpointDir = createCheckpointDirectory(checkpointsDirectory, checkpointId);
+
+		// create the checkpoint exclusive directory
+		fileSystem.mkdirs(checkpointDir);
+
+		return new FsCheckpointStorageLocation(
+						fileSystem, checkpointDir, sharedStateDirectory, taskOwnedStateDirectory);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
new file mode 100644
index 0000000..829ab9a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
@@ -0,0 +1,122 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A storage location for checkpoints on a file system.
+ */
+public class FsCheckpointStorageLocation implements CheckpointStorageLocation {
+
+	private final FileSystem fileSystem;
+
+	private final Path checkpointDirectory;
+
+	private final Path sharedStateDirectory;
+
+	private final Path taskOwnedStateDirectory;
+
+	private final Path metadataFilePath;
+
+	private final String qualifiedCheckpointDirectory;
+
+	public FsCheckpointStorageLocation(
+			FileSystem fileSystem,
+			Path checkpointDir,
+			Path sharedStateDir,
+			Path taskOwnedStateDir) {
+
+		this.fileSystem = checkNotNull(fileSystem);
+		this.checkpointDirectory = checkNotNull(checkpointDir);
+		this.sharedStateDirectory = checkNotNull(sharedStateDir);
+		this.taskOwnedStateDirectory = checkNotNull(taskOwnedStateDir);
+
+		this.metadataFilePath = new Path(checkpointDir, AbstractFsCheckpointStorage.METADATA_FILE_NAME);
+
+		this.qualifiedCheckpointDirectory = checkpointDir.makeQualified(fileSystem).toString();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public Path getCheckpointDirectory() {
+		return checkpointDirectory;
+	}
+
+	public Path getSharedStateDirectory() {
+		return sharedStateDirectory;
+	}
+
+	public Path getTaskOwnedStateDirectory() {
+		return taskOwnedStateDirectory;
+	}
+
+	public Path getMetadataFilePath() {
+		return metadataFilePath;
+	}
+
+	// ------------------------------------------------------------------------
+	//  checkpoint metadata
+	// ------------------------------------------------------------------------
+
+	@Override
+	public CheckpointStateOutputStream createMetadataOutputStream() throws IOException {
+		return new FixFileFsStateOutputStream(fileSystem, metadataFilePath);
+	}
+
+	@Override
+	public String markCheckpointAsFinished() throws IOException {
+		return qualifiedCheckpointDirectory;
+	}
+
+	@Override
+	public void disposeOnFailure() throws IOException {
+		// on a failure, no chunk in the checkpoint directory needs to be saved, so
+		// we can drop it as a whole
+		fileSystem.delete(checkpointDirectory, true);
+	}
+
+	@Override
+	public String getLocationAsPointer() {
+		return qualifiedCheckpointDirectory;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "FsCheckpointStorageLocation {" +
+				"metadataFilePath=" + metadataFilePath +
+				", taskOwnedStateDirectory=" + taskOwnedStateDirectory +
+				", sharedStateDirectory=" + sharedStateDirectory +
+				", checkpointDirectory=" + checkpointDirectory +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 2fff45a..de49552 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
@@ -28,6 +28,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
@@ -434,6 +435,12 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur
 	// ------------------------------------------------------------------------
 
 	@Override
+	public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+		checkNotNull(jobId, "jobId");
+		return new FsCheckpointStorage(getCheckpointPath(), getSavepointPath(), jobId);
+	}
+
+	@Override
 	public CheckpointStreamFactory createStreamFactory(JobID jobId, String operatorIdentifier) throws IOException {
 		return new FsCheckpointStreamFactory(getCheckpointPath(), jobId, getMinFileSizeThreshold());
 	}
@@ -447,6 +454,10 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur
 		return new FsSavepointStreamFactory(new Path(targetLocation), jobId, getMinFileSizeThreshold());
 	}
 
+	// ------------------------------------------------------------------------
+	//  state holding structures
+	// ------------------------------------------------------------------------
+
 	@Override
 	public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 			Environment env,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryBackendCheckpointStorage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryBackendCheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryBackendCheckpointStorage.java
new file mode 100644
index 0000000..3fb2627
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryBackendCheckpointStorage.java
@@ -0,0 +1,135 @@
+/*
+ * 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.memory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorage;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * An implementation of a checkpoint storage for the {@link MemoryStateBackend}.
+ * Depending on whether this is created with a checkpoint location, the setup supports
+ * durable checkpoints (durable metadata) or not.
+ */
+public class MemoryBackendCheckpointStorage extends AbstractFsCheckpointStorage {
+
+	/** The target directory for checkpoints (here metadata files only). Null, if not configured. */
+	@Nullable
+	private final Path checkpointsDirectory;
+
+	/** The file system to persist the checkpoints to. Null if this does not durably persist checkpoints. */
+	@Nullable
+	private final FileSystem fileSystem;
+
+	/**
+	 * Creates a new MemoryBackendCheckpointStorage. The storage neither persists checkpoints
+	 * in the filesystem, nor does it have a default savepoint location. The storage does support
+	 * savepoints, though, when an explicit savepoint location is passed to
+	 * {@link #initializeLocationForSavepoint(long, String)}.
+	 *
+	 * @param jobId The ID of the job writing the checkpoints.
+	 */
+	public MemoryBackendCheckpointStorage(JobID jobId) {
+		super(jobId, null);
+		checkpointsDirectory = null;
+		fileSystem = null;
+	}
+
+	/**
+	 * Creates a new MemoryBackendCheckpointStorage.
+	 *
+	 * @param jobId The ID of the job writing the checkpoints.
+	 * @param checkpointsBaseDirectory The directory to write checkpoints to. May be null,
+	 *                                 in which case this storage does not support durable persistence.
+	 * @param defaultSavepointLocation The default savepoint directory, or null, if none is set.
+	 *
+	 * @throws IOException Thrown if a checkpoint base directory is given configured and the
+	 *                     checkpoint directory cannot be created within that directory.
+	 */
+	public MemoryBackendCheckpointStorage(
+			JobID jobId,
+			@Nullable Path checkpointsBaseDirectory,
+			@Nullable Path defaultSavepointLocation) throws IOException {
+
+		super(jobId, defaultSavepointLocation);
+
+		if (checkpointsBaseDirectory == null) {
+			checkpointsDirectory = null;
+			fileSystem = null;
+		}
+		else {
+			this.fileSystem = checkpointsBaseDirectory.getFileSystem();
+			this.checkpointsDirectory = getCheckpointDirectoryForJob(checkpointsBaseDirectory, jobId);
+
+			fileSystem.mkdirs(checkpointsDirectory);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Checkpoint Storage
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsHighlyAvailableStorage() {
+		return checkpointsDirectory != null;
+	}
+
+	@Override
+	public CheckpointStorageLocation initializeLocationForCheckpoint(long checkpointId) throws IOException {
+		checkArgument(checkpointId >= 0);
+
+		if (checkpointsDirectory != null) {
+			// configured for durable metadata
+			// prepare all the paths needed for the checkpoints
+			checkState(fileSystem != null);
+
+			final Path checkpointDir = createCheckpointDirectory(checkpointsDirectory, checkpointId);
+
+			// create the checkpoint exclusive directory
+			fileSystem.mkdirs(checkpointDir);
+
+			return new PersistentMetadataCheckpointStorageLocation(fileSystem, checkpointDir);
+		}
+		else {
+			// no durable metadata - typical in IDE or test setup case
+			return new NonPersistentMetadataCheckpointStorageLocation();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return getClass().getName() + " - " +
+				(checkpointsDirectory == null ? "(not persistent)" : checkpointsDirectory) +
+				", default savepoint dir: " +
+				(getDefaultSavepointDirectory() == null ? "(none)" : getDefaultSavepointDirectory());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 2079a97..afcf9a8 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
@@ -27,6 +27,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
@@ -278,14 +279,8 @@ public class MemoryStateBackend extends AbstractFileStateBackend implements Conf
 	// ------------------------------------------------------------------------
 
 	@Override
-	public OperatorStateBackend createOperatorStateBackend(
-		Environment env,
-		String operatorIdentifier) throws Exception {
-
-		return new DefaultOperatorStateBackend(
-			env.getUserClassLoader(),
-			env.getExecutionConfig(),
-			isUsingAsynchronousSnapshots());
+	public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+		return new MemoryBackendCheckpointStorage(jobId, getCheckpointPath(), getSavepointPath());
 	}
 
 	@Override
@@ -304,10 +299,21 @@ public class MemoryStateBackend extends AbstractFileStateBackend implements Conf
 	}
 
 	// ------------------------------------------------------------------------
-	//  checkpoint state persistence
+	//  state holding structures
 	// ------------------------------------------------------------------------
 
 	@Override
+	public OperatorStateBackend createOperatorStateBackend(
+			Environment env,
+			String operatorIdentifier) throws Exception {
+
+		return new DefaultOperatorStateBackend(
+				env.getUserClassLoader(),
+				env.getExecutionConfig(),
+				isUsingAsynchronousSnapshots());
+	}
+
+	@Override
 	public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 			Environment env, JobID jobID,
 			String operatorIdentifier,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/NonPersistentMetadataCheckpointStorageLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/NonPersistentMetadataCheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/NonPersistentMetadataCheckpointStorageLocation.java
new file mode 100644
index 0000000..3baa319
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/NonPersistentMetadataCheckpointStorageLocation.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.memory;
+
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.MemoryCheckpointOutputStream;
+
+import java.io.IOException;
+
+/**
+ * A checkpoint storage location for the {@link MemoryStateBackend} in case no durable persistence
+ * for metadata has been configured.
+ */
+public class NonPersistentMetadataCheckpointStorageLocation implements CheckpointStorageLocation {
+
+	/** The external pointer returned for checkpoints that are not externally addressable. */
+	public static final String EXTERNAL_POINTER = "<checkpoint-not-externally-addressable>";
+
+	/** The maximum serialized state size for the checkpoint metadata. */
+	private static final int MAX_METADATA_STATE_SIZE = Integer.MAX_VALUE;
+
+	@Override
+	public CheckpointStateOutputStream createMetadataOutputStream() throws IOException {
+		return new MemoryCheckpointOutputStream(MAX_METADATA_STATE_SIZE);
+	}
+
+	@Override
+	public String markCheckpointAsFinished() {
+		return EXTERNAL_POINTER;
+	}
+
+	@Override
+	public void disposeOnFailure() {}
+
+	@Override
+	public String getLocationAsPointer() {
+		return PersistentMetadataCheckpointStorageLocation.LOCATION_POINTER;
+	}
+}


[03/17] flink git commit: [FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/PersistentMetadataCheckpointStorageLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/PersistentMetadataCheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/PersistentMetadataCheckpointStorageLocation.java
new file mode 100644
index 0000000..5f4b954
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/PersistentMetadataCheckpointStorageLocation.java
@@ -0,0 +1,64 @@
+/*
+ * 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.memory;
+
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation;
+
+/**
+ * A checkpoint storage location for the {@link MemoryStateBackend} when it durably
+ * persists the metadata in a file system.
+ *
+ * <p>This class inherits its behavior for metadata from the {@link FsCheckpointStorageLocation},
+ * which makes checkpoint metadata cross compatible between the two classes and hence between
+ * the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend FsStateBackend} and the
+ * {@link MemoryStateBackend}.
+ */
+public class PersistentMetadataCheckpointStorageLocation extends FsCheckpointStorageLocation {
+
+	/** The internal pointer for the {@link MemoryStateBackend}'s storage location (data inline with
+	 * state handles) that gets sent to the TaskManagers to describe this storage. */
+	static final String LOCATION_POINTER = "(embedded)";
+
+	/**
+	 * Creates a checkpoint storage persists metadata to a file system and stores state
+	 * in line in state handles with the metadata.
+	 *
+	 * @param fileSystem The file system to which the metadata will be written.
+	 * @param checkpointDir The directory where the checkpoint metadata will be written.
+	 */
+	public PersistentMetadataCheckpointStorageLocation(FileSystem fileSystem, Path checkpointDir) {
+		super(fileSystem, checkpointDir, checkpointDir, checkpointDir);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String getLocationAsPointer() {
+		return LOCATION_POINTER;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return getClass().getName() + " @ " + getCheckpointDirectory();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 0f8033d..325e955 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
@@ -39,7 +39,6 @@ import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
 import org.apache.flink.runtime.blob.{BlobServer, BlobStore}
 import org.apache.flink.runtime.checkpoint._
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
 import org.apache.flink.runtime.client._
 import org.apache.flink.runtime.clusterframework.{BootstrapTools, FlinkResourceManager}
 import org.apache.flink.runtime.clusterframework.messages._
@@ -552,31 +551,31 @@ class JobManager(
 
     case CancelJobWithSavepoint(jobId, savepointDirectory) =>
       try {
-        val targetDirectory = if (savepointDirectory != null) {
-          savepointDirectory
-        } else {
-          defaultSavepointDir
-        }
-
-        if (targetDirectory == null) {
-          log.info(s"Trying to cancel job $jobId with savepoint, but no " +
-            "savepoint directory configured.")
+        log.info(s"Trying to cancel job $jobId with savepoint to $savepointDirectory")
 
-          sender ! decorateMessage(CancellationFailure(jobId, new IllegalStateException(
-            "No savepoint directory configured. You can either specify a directory " +
-              "while cancelling via -s :targetDirectory or configure a cluster-wide " +
-              "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")))
-        } else {
-          log.info(s"Trying to cancel job $jobId with savepoint to $targetDirectory")
+        currentJobs.get(jobId) match {
+          case Some((executionGraph, _)) =>
+            val coord = executionGraph.getCheckpointCoordinator
 
-          currentJobs.get(jobId) match {
-            case Some((executionGraph, _)) =>
+            if (coord == null) {
+              sender ! decorateMessage(CancellationFailure(jobId, new IllegalStateException(
+                s"Job $jobId is not a streaming job.")))
+            }
+            else if (savepointDirectory == null &&
+                  !coord.getCheckpointStorage.hasDefaultSavepointLocation) {
+              log.info(s"Trying to cancel job $jobId with savepoint, but no " +
+                "savepoint directory configured.")
+
+              sender ! decorateMessage(CancellationFailure(jobId, new IllegalStateException(
+                "No savepoint directory configured. You can either specify a directory " +
+                  "while cancelling via -s :targetDirectory or configure a cluster-wide " +
+                  "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")))
+            } else {
               // We don't want any checkpoint between the savepoint and cancellation
-              val coord = executionGraph.getCheckpointCoordinator
               coord.stopCheckpointScheduler()
 
               // Trigger the savepoint
-              val future = coord.triggerSavepoint(System.currentTimeMillis(), targetDirectory)
+              val future = coord.triggerSavepoint(System.currentTimeMillis(), savepointDirectory)
 
               val senderRef = sender()
               future.handleAsync[Void](
@@ -608,15 +607,15 @@ class JobManager(
                   }
                 },
                 context.dispatcher)
+            }
 
-            case None =>
-              log.info(s"No job found with ID $jobId.")
-              sender ! decorateMessage(
-                CancellationFailure(
-                  jobId,
-                  new IllegalArgumentException(s"No job found with ID $jobId."))
-              )
-          }
+          case None =>
+            log.info(s"No job found with ID $jobId.")
+            sender ! decorateMessage(
+              CancellationFailure(
+                jobId,
+                new IllegalArgumentException(s"No job found with ID $jobId."))
+            )
         }
       } catch {
         case t: Throwable =>
@@ -746,25 +745,28 @@ class JobManager(
         case Some((graph, _)) =>
           val checkpointCoordinator = graph.getCheckpointCoordinator()
 
-          if (checkpointCoordinator != null) {
+          if (checkpointCoordinator == null) {
+            sender ! decorateMessage(TriggerSavepointFailure(jobId, new IllegalStateException(
+              s"Job $jobId is not a streaming job.")))
+          }
+          else if (savepointDirectory.isEmpty &&
+                !checkpointCoordinator.getCheckpointStorage.hasDefaultSavepointLocation) {
+            log.info(s"Trying to trigger a savepoint, but no savepoint directory configured.")
+
+            sender ! decorateMessage(TriggerSavepointFailure(jobId, new IllegalStateException(
+              "No savepoint directory configured. You can either specify a directory " +
+                "when triggering the savepoint via -s :targetDirectory or configure a " +
+                "cluster-/application-wide default via key '" +
+                CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")))
+          } else {
             // Immutable copy for the future
             val senderRef = sender()
             try {
-              val targetDirectory : String = savepointDirectory.getOrElse(
-                flinkConfiguration.getString(CheckpointingOptions.SAVEPOINT_DIRECTORY))
-
-              if (targetDirectory == null) {
-                throw new IllegalStateException("No savepoint directory configured. " +
-                  "You can either specify a directory when triggering this savepoint or " +
-                  "configure a cluster-wide default via key '" +
-                  CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")
-              }
-
               // Do this async, because checkpoint coordinator operations can
               // contain blocking calls to the state backend or ZooKeeper.
               val savepointFuture = checkpointCoordinator.triggerSavepoint(
                 System.currentTimeMillis(),
-                targetDirectory)
+                savepointDirectory.orNull)
 
               savepointFuture.handleAsync[Void](
                 new BiFunction[CompletedCheckpoint, Throwable, Void] {
@@ -794,10 +796,6 @@ class JobManager(
                 senderRef ! TriggerSavepointFailure(jobId, new Exception(
                   "Failed to trigger savepoint", e))
             }
-          } else {
-            sender() ! TriggerSavepointFailure(jobId, new IllegalStateException(
-              "Checkpointing disabled. You can enable it via the execution environment of " +
-                "your job."))
           }
 
         case None =>
@@ -809,19 +807,13 @@ class JobManager(
       future {
         try {
           log.info(s"Disposing savepoint at '$savepointPath'.")
-          //TODO user code class loader ?
-          // (has not been used so far and new savepoints can simply be deleted by file)
-          val savepoint = SavepointStore.loadSavepoint(
-            savepointPath,
-            Thread.currentThread().getContextClassLoader)
-
-          log.debug(s"$savepoint")
 
-          // Dispose checkpoint state
-          savepoint.dispose()
+          // there is a corner case issue with Flink 1.1 savepoints, which may contain
+          // user-defined state handles. however, it should work for all the standard cases,
+          // where the mem/fs/rocks state backends were used
+          val classLoader = Thread.currentThread().getContextClassLoader
 
-          // Remove the header file
-          SavepointStore.removeSavepointFile(savepointPath)
+          Checkpoints.disposeSavepoint(savepointPath, flinkConfiguration, classLoader, log.logger)
 
           senderRef ! DisposeSavepointSuccess
         } catch {

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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
deleted file mode 100644
index f55e0d3..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.checkpoint;
-
-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.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
-import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
-import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.filesystem.FileStateHandle;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * 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 final 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();
-		final FsStateBackend stateBackend = new FsStateBackend(checkpointDir.toURI());
-
-		// 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(),
-			stateBackend,
-			Executors.directExecutor(),
-			SharedStateRegistry.DEFAULT_FACTORY);
-
-		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) {
-			for (OperatorID operatorID : vertex.getJobVertex().getOperatorIDs()) {
-				assertEquals(checkpoint.getOperatorStates().get(operatorID), loaded.getOperatorStates().get(operatorID));
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 add7447..cbfe0ed 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
@@ -24,7 +24,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
@@ -72,13 +71,12 @@ public class CheckpointCoordinatorFailureTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[]{vertex},
 			new ExecutionVertex[]{vertex},
 			new ExecutionVertex[]{vertex},
 			new StandaloneCheckpointIDCounter(),
 			new FailingCompletedCheckpointStore(),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
index e53bf3a..d583a0d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
@@ -26,12 +26,13 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.state.EmptyStreamStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
-
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+
 import org.junit.Test;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -229,9 +230,9 @@ public class CheckpointCoordinatorMasterHooksTest {
 				jid, checkpointId, 123L, 125L,
 				Collections.<OperatorID, OperatorState>emptyMap(),
 				masterHookStates,
-				CheckpointProperties.forStandardCheckpoint(),
-				null,
-				null);
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+				new EmptyStreamStateHandle(),
+				"pointer");
 
 		final ExecutionAttemptID execId = new ExecutionAttemptID();
 		final ExecutionVertex ackVertex = mockExecutionVertex(execId);
@@ -283,9 +284,9 @@ public class CheckpointCoordinatorMasterHooksTest {
 				jid, checkpointId, 123L, 125L,
 				Collections.<OperatorID, OperatorState>emptyMap(),
 				masterHookStates,
-				CheckpointProperties.forStandardCheckpoint(),
-				null,
-				null);
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+				new EmptyStreamStateHandle(),
+				"pointer");
 
 		final ExecutionAttemptID execId = new ExecutionAttemptID();
 		final ExecutionVertex ackVertex = mockExecutionVertex(execId);
@@ -396,13 +397,12 @@ public class CheckpointCoordinatorMasterHooksTest {
 				600000L,
 				0L,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[0],
 				ackVertices,
 				new ExecutionVertex[0],
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(10),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 609e91c..9f9659a 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
@@ -32,10 +32,10 @@ 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.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.state.ChainedStateHandle;
+import org.apache.flink.runtime.state.EmptyStreamStateHandle;
 import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
@@ -135,13 +135,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 				new ExecutionVertex[] { ackVertex1, ackVertex2 },
 				new ExecutionVertex[] {},
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -197,13 +196,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 				new ExecutionVertex[] { ackVertex1, ackVertex2 },
 				new ExecutionVertex[] {},
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -250,13 +248,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 				new ExecutionVertex[] { ackVertex1, ackVertex2 },
 				new ExecutionVertex[] {},
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -304,13 +301,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -408,13 +404,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -529,13 +524,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new ExecutionVertex[] { vertex1, vertex2 },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -698,13 +692,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 				new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -830,13 +823,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 				new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(10),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -996,13 +988,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex1, ackVertex2 },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1075,13 +1066,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex1, ackVertex2 },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1140,13 +1130,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			20000L,
 			0L,
 			1,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { triggerVertex },
 			new ExecutionVertex[] {triggerVertex, ackVertex1, ackVertex2},
 			new ExecutionVertex[0],
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -1274,13 +1263,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200000,    // timeout is very long (200 s)
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1365,13 +1353,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200_000,     // timeout is very long (200 s)
 				delay,       // 50 ms delay between checkpoints
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { vertex },
 				new ExecutionVertex[] { vertex },
 				new ExecutionVertex[] { vertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				"dummy-path",
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1440,13 +1427,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { vertex1, vertex2 },
 			new ExecutionVertex[] { vertex1, vertex2 },
 			new ExecutionVertex[] { vertex1, vertex2 },
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -1593,13 +1579,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { vertex1, vertex2 },
 			new ExecutionVertex[] { vertex1, vertex2 },
 			new ExecutionVertex[] { vertex1, vertex2 },
 			counter,
 			new StandaloneCompletedCheckpointStore(10),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -1695,13 +1680,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200000,    // timeout is very long (200 s)
 				0L,        // no extra delay
 				maxConcurrentAttempts,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1770,13 +1754,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200000,    // timeout is very long (200 s)
 				0L,        // no extra delay
 				maxConcurrentAttempts, // max two concurrent checkpoints
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1848,13 +1831,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				200000,    // timeout is very long (200 s)
 				0L,        // no extra delay
 				2, // max two concurrent checkpoints
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { triggerVertex },
 				new ExecutionVertex[] { ackVertex },
 				new ExecutionVertex[] { commitVertex },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(2),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -1902,13 +1884,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			200000,
 			0L,
 			1, // max one checkpoint at a time => should not affect savepoints
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			checkpointIDCounter,
 			new StandaloneCompletedCheckpointStore(2),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -1957,13 +1938,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			200000,
 			100000000L, // very long min delay => should not affect savepoints
 			1,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(2),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -2021,13 +2001,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			new StandaloneCheckpointIDCounter(),
 			store,
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -2137,13 +2116,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -2285,13 +2263,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -2436,7 +2413,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 	 */
 	public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception {
 
-		/**
+		/*
 		 * Old topology
 		 * CHAIN(op1 -> op2) * parallelism1 -> CHAIN(op3 -> op4) * parallelism2
 		 */
@@ -2510,7 +2487,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			}
 		}
 
-		/**
+		/*
 		 * New topology
 		 * CHAIN(op5 -> op1 -> op2) * newParallelism1 -> CHAIN(op3 -> op6) * newParallelism2
 		 */
@@ -2557,9 +2534,9 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				System.currentTimeMillis() + 3000,
 				operatorStates,
 				Collections.<MasterState>emptyList(),
-				CheckpointProperties.forStandardCheckpoint(),
-				null,
-				null);
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+				new EmptyStreamStateHandle(),
+				"some/mock/pointer");
 
 		when(standaloneCompletedCheckpointStore.getLatestCheckpoint()).thenReturn(completedCheckpoint);
 
@@ -2570,13 +2547,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			newJobVertex1.getTaskVertices(),
 			newJobVertex1.getTaskVertices(),
 			newJobVertex1.getTaskVertices(),
 			new StandaloneCheckpointIDCounter(),
 			standaloneCompletedCheckpointStore,
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -2719,13 +2695,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				600000,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.externalizeCheckpoints(true),
+				CheckpointRetentionPolicy.RETAIN_ON_FAILURE,
 				new ExecutionVertex[] { vertex1 },
 				new ExecutionVertex[] { vertex1 },
 				new ExecutionVertex[] { vertex1 },
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				"fake-directory",
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -2734,7 +2709,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 			for (PendingCheckpoint checkpoint : coord.getPendingCheckpoints().values()) {
 				CheckpointProperties props = checkpoint.getProps();
-				CheckpointProperties expected = CheckpointProperties.forExternalizedCheckpoint(true);
+				CheckpointProperties expected = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE);
 
 				assertEquals(expected, props);
 			}
@@ -3196,13 +3171,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			new ExecutionVertex[] { vertex1 },
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -3210,7 +3184,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 		// Periodic
 		CheckpointTriggerResult triggerResult = coord.triggerCheckpoint(
 				System.currentTimeMillis(),
-				CheckpointProperties.forStandardCheckpoint(),
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 				null,
 				true);
 
@@ -3220,7 +3194,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 		// Not periodic
 		triggerResult = coord.triggerCheckpoint(
 				System.currentTimeMillis(),
-				CheckpointProperties.forStandardCheckpoint(),
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 				null,
 				false);
 
@@ -3376,13 +3350,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -3397,7 +3370,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 		assertTrue(coord.triggerCheckpoint(timestamp, false));
 
 		verify(tracker, times(1))
-			.reportPendingCheckpoint(eq(1L), eq(timestamp), eq(CheckpointProperties.forStandardCheckpoint()));
+			.reportPendingCheckpoint(eq(1L), eq(timestamp), eq(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION)));
 	}
 
 	/**
@@ -3416,13 +3389,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			new StandaloneCheckpointIDCounter(),
 			store,
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -3434,9 +3406,9 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			0,
 			Collections.<OperatorID, OperatorState>emptyMap(),
 			Collections.<MasterState>emptyList(),
-			CheckpointProperties.forStandardCheckpoint(),
-			null,
-			null));
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			new EmptyStreamStateHandle(),
+			"some/dummy/pointer"));
 
 		CheckpointStatsTracker tracker = mock(CheckpointStatsTracker.class);
 		coord.setCheckpointStatsTracker(tracker);
@@ -3474,13 +3446,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000L,
 			0L,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			new ExecutionVertex[]{vertex1},
 			checkpointIDCounter,
 			completedCheckpointStore,
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -3568,13 +3539,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			arrayExecutionVertices,
 			new StandaloneCheckpointIDCounter(),
 			store,
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 				deleteExecutor -> {

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointExternalResumeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointExternalResumeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointExternalResumeTest.java
new file mode 100644
index 0000000..86eec77
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointExternalResumeTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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 org.apache.flink.api.common.JobID;
+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.OperatorID;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * 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 CheckpointExternalResumeTest {
+
+	@Rule
+	public final 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 File checkpointDir = tmp.newFolder();
+		final FsStateBackend stateBackend = new FsStateBackend(checkpointDir.toURI());
+
+		// 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,
+			CheckpointRetentionPolicy.RETAIN_ON_FAILURE,
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new ExecutionVertex[] { vertex1, vertex2 },
+			new StandaloneCheckpointIDCounter(),
+			new StandaloneCompletedCheckpointStore(1),
+			stateBackend,
+			Executors.directExecutor(),
+			SharedStateRegistry.DEFAULT_FACTORY);
+
+		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 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());
+		FileStateHandle fsHandle = (FileStateHandle) checkpoint.getMetadataHandle();
+		assertTrue(new File(fsHandle.getFilePath().getPath()).exists());
+	}
+
+	private void verifyExternalizedCheckpointRestore(
+			CompletedCheckpoint checkpoint,
+			Map<JobVertexID, ExecutionJobVertex> jobVertices,
+			ExecutionVertex... vertices) throws IOException {
+
+		String pointer = checkpoint.getExternalPointer();
+		StreamStateHandle metadataHandle = new FsStateBackend(tmp.getRoot().toURI()).resolveCheckpoint(pointer);
+
+		CompletedCheckpoint loaded = Checkpoints.loadAndValidateCheckpoint(
+				checkpoint.getJobId(),
+				jobVertices,
+				checkpoint.getExternalPointer(),
+				metadataHandle,
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		for (ExecutionVertex vertex : vertices) {
+			for (OperatorID operatorID : vertex.getJobVertex().getOperatorIDs()) {
+				assertEquals(checkpoint.getOperatorStates().get(operatorID), loaded.getOperatorStates().get(operatorID));
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java
new file mode 100644
index 0000000..ee60fc7
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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 org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * A test that checks that checkpoint metadata loading works properly, including validation
+ * of resumed state and dropped state.
+ */
+public class CheckpointMetadataLoadingTest {
+
+	@Rule
+	public final TemporaryFolder tmpFolder = new TemporaryFolder();
+
+	/**
+	 * Tests loading and validation of savepoints with correct setup,
+	 * parallelism mismatch, and a missing task.
+	 */
+	@Test
+	public void testLoadAndValidateSavepoint() throws Exception {
+		File tmp = tmpFolder.newFolder();
+
+		int parallelism = 128128;
+		long checkpointId = Integer.MAX_VALUE + 123123L;
+		JobVertexID jobVertexID = new JobVertexID();
+		OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID);
+
+		OperatorSubtaskState subtaskState = new OperatorSubtaskState(
+				new OperatorStateHandle(Collections.emptyMap(), new ByteStreamStateHandle("testHandler", new byte[0])),
+				null,
+				null,
+				null);
+
+		OperatorState state = new OperatorState(operatorID, parallelism, parallelism);
+		state.putState(0, subtaskState);
+
+		Map<OperatorID, OperatorState> taskStates = new HashMap<>();
+		taskStates.put(operatorID, state);
+
+		JobID jobId = new JobID();
+
+		// Store savepoint
+		final SavepointV2 savepoint = new SavepointV2(checkpointId, taskStates.values(), Collections.emptyList());
+		final StreamStateHandle serializedMetadata;
+
+		try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+			Checkpoints.storeCheckpointMetadata(savepoint, os);
+			serializedMetadata = new ByteStreamStateHandle("checkpoint", os.toByteArray());
+		}
+
+		ExecutionJobVertex vertex = mock(ExecutionJobVertex.class);
+		when(vertex.getParallelism()).thenReturn(parallelism);
+		when(vertex.getMaxParallelism()).thenReturn(parallelism);
+		when(vertex.getOperatorIDs()).thenReturn(Collections.singletonList(operatorID));
+
+		Map<JobVertexID, ExecutionJobVertex> tasks = new HashMap<>();
+		tasks.put(jobVertexID, vertex);
+
+		ClassLoader ucl = Thread.currentThread().getContextClassLoader();
+
+		// 1) Load and validate: everything correct
+		CompletedCheckpoint loaded = Checkpoints.loadAndValidateCheckpoint(jobId, tasks, "fake/path", serializedMetadata, ucl, false);
+
+		assertEquals(jobId, loaded.getJobId());
+		assertEquals(checkpointId, loaded.getCheckpointID());
+
+		// 2) Load and validate: max parallelism mismatch
+		when(vertex.getMaxParallelism()).thenReturn(222);
+		when(vertex.isMaxParallelismConfigured()).thenReturn(true);
+
+		try {
+			Checkpoints.loadAndValidateCheckpoint(jobId, tasks, "fake/path", serializedMetadata, ucl, false);
+			fail("Did not throw expected Exception");
+		} catch (IllegalStateException expected) {
+			assertTrue(expected.getMessage().contains("Max parallelism mismatch"));
+		}
+
+		// 3) Load and validate: missing vertex
+		assertNotNull(tasks.remove(jobVertexID));
+
+		try {
+			Checkpoints.loadAndValidateCheckpoint(jobId, tasks, "fake/path", serializedMetadata, ucl, false);
+			fail("Did not throw expected Exception");
+		} catch (IllegalStateException expected) {
+			assertTrue(expected.getMessage().contains("allowNonRestoredState"));
+		}
+
+		// 4) Load and validate: ignore missing vertex
+		Checkpoints.loadAndValidateCheckpoint(jobId, tasks, "fake/path", serializedMetadata, ucl, true);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java
index a0509c4..c17172b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java
@@ -31,40 +31,22 @@ import static org.junit.Assert.assertTrue;
 public class CheckpointPropertiesTest {
 
 	/**
-	 * Tests the default checkpoint properties.
-	 */
-	@Test
-	public void testCheckpointProperties() {
-		CheckpointProperties props = CheckpointProperties.forStandardCheckpoint();
-
-		assertFalse(props.forceCheckpoint());
-		assertFalse(props.externalizeCheckpoint());
-		assertTrue(props.discardOnSubsumed());
-		assertTrue(props.discardOnJobFinished());
-		assertTrue(props.discardOnJobCancelled());
-		assertTrue(props.discardOnJobFailed());
-		assertTrue(props.discardOnJobSuspended());
-	}
-
-	/**
 	 * Tests the external checkpoints properties.
 	 */
 	@Test
-	public void testExternalizedCheckpointProperties() {
-		CheckpointProperties props = CheckpointProperties.forExternalizedCheckpoint(true);
+	public void testCheckpointProperties() {
+		CheckpointProperties props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE);
 
 		assertFalse(props.forceCheckpoint());
-		assertTrue(props.externalizeCheckpoint());
 		assertTrue(props.discardOnSubsumed());
 		assertTrue(props.discardOnJobFinished());
 		assertTrue(props.discardOnJobCancelled());
 		assertFalse(props.discardOnJobFailed());
 		assertTrue(props.discardOnJobSuspended());
 
-		props = CheckpointProperties.forExternalizedCheckpoint(false);
+		props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION);
 
 		assertFalse(props.forceCheckpoint());
-		assertTrue(props.externalizeCheckpoint());
 		assertTrue(props.discardOnSubsumed());
 		assertTrue(props.discardOnJobFinished());
 		assertFalse(props.discardOnJobCancelled());
@@ -77,10 +59,9 @@ public class CheckpointPropertiesTest {
 	 */
 	@Test
 	public void testSavepointProperties() {
-		CheckpointProperties props = CheckpointProperties.forStandardSavepoint();
+		CheckpointProperties props = CheckpointProperties.forSavepoint();
 
 		assertTrue(props.forceCheckpoint());
-		assertTrue(props.externalizeCheckpoint());
 		assertFalse(props.discardOnSubsumed());
 		assertFalse(props.discardOnJobFinished());
 		assertFalse(props.discardOnJobCancelled());
@@ -94,22 +75,17 @@ public class CheckpointPropertiesTest {
 	@Test
 	public void testIsSavepoint() throws Exception {
 		{
-			CheckpointProperties props = CheckpointProperties.forStandardCheckpoint();
-			assertFalse(props.isSavepoint());
-		}
-
-		{
-			CheckpointProperties props = CheckpointProperties.forExternalizedCheckpoint(true);
+			CheckpointProperties props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE);
 			assertFalse(props.isSavepoint());
 		}
 
 		{
-			CheckpointProperties props = CheckpointProperties.forExternalizedCheckpoint(false);
+			CheckpointProperties props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION);
 			assertFalse(props.isSavepoint());
 		}
 
 		{
-			CheckpointProperties props = CheckpointProperties.forStandardSavepoint();
+			CheckpointProperties props = CheckpointProperties.forSavepoint();
 			assertTrue(props.isSavepoint());
 
 			CheckpointProperties deserializedCheckpointProperties =

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java
index e98efc2..b8a2a54 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.runtime.checkpoint;
 
-import java.io.IOException;
-import javax.annotation.Nullable;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -34,21 +32,24 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
 import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
 
+import javax.annotation.Nullable;
+import java.io.IOException;
 import java.io.Serializable;
 import java.net.URL;
 import java.net.URLClassLoader;
@@ -83,7 +84,7 @@ public class CheckpointSettingsSerializableTest extends TestLogger {
 					10000L,
 					0L,
 					1,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					true),
 				new SerializedValue<StateBackend>(new CustomStateBackend(outOfClassPath)),
 				serHooks);
@@ -142,6 +143,7 @@ public class CheckpointSettingsSerializableTest extends TestLogger {
 		/**
 		 * Simulate a custom option that is not in the normal classpath.
 		 */
+		@SuppressWarnings("unused")
 		private Serializable customOption;
 
 		public CustomStateBackend(Serializable customOption) {
@@ -149,9 +151,19 @@ public class CheckpointSettingsSerializableTest extends TestLogger {
 		}
 
 		@Override
+		public StreamStateHandle resolveCheckpoint(String pointer) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException {
+			return mock(CheckpointStorage.class);
+		}
+
+		@Override
 		public CheckpointStreamFactory createStreamFactory(
 			JobID jobId, String operatorIdentifier) throws IOException {
-			return null;
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
@@ -159,7 +171,7 @@ public class CheckpointSettingsSerializableTest extends TestLogger {
 			JobID jobId,
 			String operatorIdentifier,
 			@Nullable String targetLocation) throws IOException {
-			return null;
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
@@ -171,13 +183,13 @@ public class CheckpointSettingsSerializableTest extends TestLogger {
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
 			TaskKvStateRegistry kvStateRegistry) throws Exception {
-			return null;
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
 		public OperatorStateBackend createOperatorStateBackend(
 			Environment env, String operatorIdentifier) throws Exception {
-			return null;
+			throw new UnsupportedOperationException();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
index df2d37a..ae354bb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
@@ -27,8 +27,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.state.EmptyStreamStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
@@ -101,13 +101,12 @@ public class CheckpointStateRestoreTest {
 				200000L,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
 				new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
 				new ExecutionVertex[0],
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -179,13 +178,12 @@ public class CheckpointStateRestoreTest {
 				200000L,
 				0,
 				Integer.MAX_VALUE,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				new ExecutionVertex[] { mock(ExecutionVertex.class) },
 				new ExecutionVertex[] { mock(ExecutionVertex.class) },
 				new ExecutionVertex[0],
 				new StandaloneCheckpointIDCounter(),
 				new StandaloneCompletedCheckpointStore(1),
-				null,
 				new MemoryStateBackend(),
 				Executors.directExecutor(),
 				SharedStateRegistry.DEFAULT_FACTORY);
@@ -207,7 +205,7 @@ public class CheckpointStateRestoreTest {
 	/**
 	 * Tests that the allow non restored state flag is correctly handled.
 	 *
-	 * The flag only applies for state that is part of the checkpoint.
+	 * <p>The flag only applies for state that is part of the checkpoint.
 	 */
 	@Test
 	public void testNonRestoredState() throws Exception {
@@ -238,13 +236,12 @@ public class CheckpointStateRestoreTest {
 			Integer.MAX_VALUE,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			new ExecutionVertex[] {},
 			new ExecutionVertex[] {},
 			new ExecutionVertex[] {},
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
@@ -266,9 +263,9 @@ public class CheckpointStateRestoreTest {
 			2,
 			new HashMap<>(checkpointTaskStates),
 			Collections.<MasterState>emptyList(),
-			CheckpointProperties.forStandardCheckpoint(),
-			null,
-			null);
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			new EmptyStreamStateHandle(),
+			"<pointer>");
 
 		coord.getCheckpointStore().addCheckpoint(checkpoint);
 
@@ -294,9 +291,9 @@ public class CheckpointStateRestoreTest {
 			3,
 			new HashMap<>(checkpointTaskStates),
 			Collections.<MasterState>emptyList(),
-			CheckpointProperties.forStandardCheckpoint(),
-			null,
-			null);
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			new EmptyStreamStateHandle(),
+			"pointer");
 
 		coord.getCheckpointStore().addCheckpoint(checkpoint);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsHistoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsHistoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsHistoryTest.java
index 95a31d4..a410d26 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsHistoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsHistoryTest.java
@@ -175,23 +175,23 @@ public class CheckpointStatsHistoryTest {
 		PendingCheckpointStats pending = mock(PendingCheckpointStats.class);
 		when(pending.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
 		when(pending.getCheckpointId()).thenReturn(checkpointId);
-		when(pending.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(pending.getProperties()).thenReturn(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		return pending;
 	}
 
 	private CompletedCheckpointStats createCompletedCheckpointStats(long checkpointId) {
 		CompletedCheckpointStats completed = mock(CompletedCheckpointStats.class);
 		when(completed.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
-		when(completed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(completed.getProperties()).thenReturn(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		when(completed.getCheckpointId()).thenReturn(checkpointId);
-		when(completed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(completed.getProperties()).thenReturn(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		return completed;
 	}
 
 	private FailedCheckpointStats createFailedCheckpointStats(long checkpointId) {
 		FailedCheckpointStats failed = mock(FailedCheckpointStats.class);
 		when(failed.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
-		when(failed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(failed.getProperties()).thenReturn(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		when(failed.getCheckpointId()).thenReturn(checkpointId);
 		return failed;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsSnapshotTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsSnapshotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsSnapshotTest.java
index 6500369..b4bb88a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsSnapshotTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsSnapshotTest.java
@@ -45,7 +45,11 @@ public class CheckpointStatsSnapshotTest {
 		summary.updateSummary(createCompletedCheckpointsStats(2221, 3333, 9122));
 
 		CheckpointStatsHistory history = new CheckpointStatsHistory(1);
-		RestoredCheckpointStats restored = new RestoredCheckpointStats(1, CheckpointProperties.forStandardCheckpoint(), 99119, null);
+		RestoredCheckpointStats restored = new RestoredCheckpointStats(
+				1,
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+				99119,
+				null);
 
 		CheckpointStatsSnapshot snapshot = new CheckpointStatsSnapshot(
 			counts,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 0d19cd5..82dcd02 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
@@ -38,9 +38,9 @@ import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
+
 import org.junit.Test;
 
 public class CheckpointStatsTrackerTest {
@@ -63,7 +63,7 @@ public class CheckpointStatsTrackerTest {
 				19191992L,
 				191929L,
 				123,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				false
 			),
 			null);
@@ -98,7 +98,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats pending = tracker.reportPendingCheckpoint(
 			0,
 			1,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		pending.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(0));
 		pending.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(1));
@@ -147,7 +147,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats completed1 = tracker.reportPendingCheckpoint(
 			0,
 			1,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		completed1.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(0));
 		completed1.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(1));
@@ -159,7 +159,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats failed = tracker.reportPendingCheckpoint(
 			1,
 			1,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		failed.reportFailedCheckpoint(12, null);
 
@@ -167,7 +167,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats savepoint = tracker.reportPendingCheckpoint(
 			2,
 			1,
-			CheckpointProperties.forStandardSavepoint());
+			CheckpointProperties.forSavepoint());
 
 		savepoint.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(0));
 		savepoint.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(1));
@@ -179,9 +179,9 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats inProgress = tracker.reportPendingCheckpoint(
 			3,
 			1,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
-		RestoredCheckpointStats restored = new RestoredCheckpointStats(81, CheckpointProperties.forStandardCheckpoint(), 123, null);
+		RestoredCheckpointStats restored = new RestoredCheckpointStats(81, CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), 123, null);
 		tracker.reportRestoredCheckpoint(restored);
 
 		CheckpointStatsSnapshot snapshot = tracker.createSnapshot();
@@ -254,7 +254,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats pending = tracker.reportPendingCheckpoint(
 			0,
 			1,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		pending.reportSubtaskStats(jobVertex.getJobVertexId(), createSubtaskStats(0));
 
@@ -270,7 +270,7 @@ public class CheckpointStatsTrackerTest {
 		assertNotEquals(snapshot2, snapshot3);
 
 		// Restore operation => new snapshot
-		tracker.reportRestoredCheckpoint(new RestoredCheckpointStats(12, CheckpointProperties.forStandardCheckpoint(), 12, null));
+		tracker.reportRestoredCheckpoint(new RestoredCheckpointStats(12, CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), 12, null));
 
 		CheckpointStatsSnapshot snapshot4 = tracker.createSnapshot();
 		assertNotEquals(snapshot3, snapshot4);
@@ -373,7 +373,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats pending = stats.reportPendingCheckpoint(
 			0,
 			0,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		// Check counts
 		assertEquals(Long.valueOf(1), numCheckpoints.getValue());
@@ -415,7 +415,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats nextPending = stats.reportPendingCheckpoint(
 			1,
 			11,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		long failureTimestamp = 1230123L;
 		nextPending.reportFailedCheckpoint(failureTimestamp, null);
@@ -430,7 +430,7 @@ public class CheckpointStatsTrackerTest {
 		long restoreTimestamp = 183419283L;
 		RestoredCheckpointStats restored = new RestoredCheckpointStats(
 			1,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			restoreTimestamp,
 			null);
 		stats.reportRestoredCheckpoint(restored);
@@ -446,7 +446,7 @@ public class CheckpointStatsTrackerTest {
 		PendingCheckpointStats thirdPending = stats.reportPendingCheckpoint(
 			2,
 			5000,
-			CheckpointProperties.forStandardCheckpoint());
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 
 		thirdPending.reportSubtaskStats(jobVertex.getJobVertexId(), subtaskStats);
 		thirdPending.reportCompletedCheckpoint(null);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummaryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummaryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummaryTest.java
index dd9c2c8..2910765 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummaryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummaryTest.java
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+
 import org.junit.Test;
 
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -93,7 +93,7 @@ public class CompletedCheckpointStatsSummaryTest {
 		return new CompletedCheckpointStats(
 			checkpointId,
 			triggerTimestamp,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			1,
 			taskStats,
 			1,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 320dc2d..d113aa1 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.checkpoint;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.EmptyStreamStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
@@ -196,7 +197,7 @@ public abstract class CompletedCheckpointStoreTest extends TestLogger {
 		SharedStateRegistry sharedStateRegistry) throws IOException {
 
 		int numberOfStates = 4;
-		CheckpointProperties props = CheckpointProperties.forStandardCheckpoint();
+		CheckpointProperties props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION);
 
 		OperatorID operatorID = new OperatorID();
 
@@ -256,7 +257,8 @@ public abstract class CompletedCheckpointStoreTest extends TestLogger {
 			long timestamp,
 			Map<OperatorID, OperatorState> operatorGroupState,
 			CheckpointProperties props) {
-			super(jobId, checkpointId, timestamp, Long.MAX_VALUE, operatorGroupState, null, props, null, null);
+			super(jobId, checkpointId, timestamp, Long.MAX_VALUE, operatorGroupState, null, props,
+					new EmptyStreamStateHandle(), "<pointer");
 		}
 
 		@Override


[06/17] flink git commit: [hotfix] [rocksdb] Clean up RocksDB state backend code

Posted by se...@apache.org.
[hotfix] [rocksdb] Clean up RocksDB state backend code

  - arrange variables to properly express configuration (client side) versus runtime (task manager side)
  - make all runtime-only fields properly transient
  - fix confusing variable name for local directories


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

Branch: refs/heads/master
Commit: 1931993bdc1d294a0eb9e1ad727f737cf64fe150
Parents: fa03e78
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Oct 26 14:55:28 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../streaming/state/RocksDBStateBackend.java    | 73 ++++++++++----------
 1 file changed, 37 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1931993b/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 79771f3..a6552bc 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
@@ -36,9 +36,12 @@ import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.NativeLibraryLoader;
 import org.rocksdb.RocksDB;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -49,7 +52,7 @@ import java.util.List;
 import java.util.Random;
 import java.util.UUID;
 
-import static java.util.Objects.requireNonNull;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * A State Backend that stores its state in {@code RocksDB}. This state backend can
@@ -76,42 +79,40 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	private static boolean rocksDbInitialized = false;
 
 	// ------------------------------------------------------------------------
-	//  Static configuration values
-	// ------------------------------------------------------------------------
+
+	// -- configuration values, set in the application / configuration
 
 	/** The state backend that we use for creating checkpoint streams. */
 	private final AbstractStateBackend checkpointStreamBackend;
 
-	/** Operator identifier that is used to uniquify the RocksDB storage path. */
-	private String operatorIdentifier;
-
-	/** JobID for uniquifying backup paths. */
-	private JobID jobId;
-
-	// DB storage directories
-
 	/** Base paths for RocksDB directory, as configured. May be null. */
-	private Path[] configuredDbBasePaths;
-
-	/** Base paths for RocksDB directory, as initialized. */
-	private File[] initializedDbBasePaths;
-
-	private int nextDirectory;
-
-	// RocksDB options
+	@Nullable
+	private Path[] localRocksDbDirectories;
 
 	/** The pre-configured option settings. */
 	private PredefinedOptions predefinedOptions = PredefinedOptions.DEFAULT;
 
 	/** The options factory to create the RocksDB options in the cluster. */
+	@Nullable
 	private OptionsFactory optionsFactory;
 
-	/** Whether we already lazily initialized our local storage directories. */
-	private transient boolean isInitialized = false;
-
 	/** True if incremental checkpointing is enabled. */
 	private boolean enableIncrementalCheckpointing;
 
+	// -- runtime values, set on TaskManager when initializing / using the backend
+
+	/** Base paths for RocksDB directory, as initialized. */
+	private transient File[] initializedDbBasePaths;
+
+	/** JobID for uniquifying backup paths. */
+	private transient JobID jobId;
+
+	private transient int nextDirectory;
+	
+	/** Whether we already lazily initialized our local storage directories. */
+	private transient boolean isInitialized;
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Creates a new {@code RocksDBStateBackend} that stores its checkpoint data in the
@@ -190,7 +191,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * @param checkpointStreamBackend The backend to store the
 	 */
 	public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend) {
-		this.checkpointStreamBackend = requireNonNull(checkpointStreamBackend);
+		this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend);
 	}
 
 	/**
@@ -202,10 +203,10 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * {@link AbstractStateBackend#createStreamFactory(JobID, String) checkpoint stream}.
 	 *
 	 * @param checkpointStreamBackend The backend to store the
-	 * @param enableIncrementalCheckpointing True if incremental checkpointing is enabled
+	 * @param enableIncrementalCheckpointing True if incremental checkponting is enabled
 	 */
 	public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend, boolean enableIncrementalCheckpointing) {
-		this.checkpointStreamBackend = requireNonNull(checkpointStreamBackend);
+		this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend);
 		this.enableIncrementalCheckpointing = enableIncrementalCheckpointing;
 	}
 
@@ -221,19 +222,18 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 			return;
 		}
 
-		this.operatorIdentifier = operatorIdentifier.replace(" ", "");
 		this.jobId = env.getJobID();
 
 		// initialize the paths where the local RocksDB files should be stored
-		if (configuredDbBasePaths == null) {
+		if (localRocksDbDirectories == null) {
 			// initialize from the temp directories
 			initializedDbBasePaths = env.getIOManager().getSpillingDirectories();
 		}
 		else {
-			List<File> dirs = new ArrayList<>(configuredDbBasePaths.length);
+			List<File> dirs = new ArrayList<>(localRocksDbDirectories.length);
 			String errorMessage = "";
 
-			for (Path path : configuredDbBasePaths) {
+			for (Path path : localRocksDbDirectories) {
 				File f = new File(path.toUri().getPath());
 				File testDir = new File(f, UUID.randomUUID().toString());
 				if (!testDir.mkdirs()) {
@@ -244,6 +244,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 				} else {
 					dirs.add(f);
 				}
+				//noinspection ResultOfMethodCallIgnored
 				testDir.delete();
 			}
 
@@ -349,7 +350,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 */
 	public void setDbStoragePaths(String... paths) {
 		if (paths == null) {
-			configuredDbBasePaths = null;
+			localRocksDbDirectories = null;
 		}
 		else if (paths.length == 0) {
 			throw new IllegalArgumentException("empty paths");
@@ -369,7 +370,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 				}
 			}
 
-			configuredDbBasePaths = pp;
+			localRocksDbDirectories = pp;
 		}
 	}
 
@@ -378,12 +379,12 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * @return The configured DB storage paths, or null, if none were configured.
 	 */
 	public String[] getDbStoragePaths() {
-		if (configuredDbBasePaths == null) {
+		if (localRocksDbDirectories == null) {
 			return null;
 		} else {
-			String[] paths = new String[configuredDbBasePaths.length];
+			String[] paths = new String[localRocksDbDirectories.length];
 			for (int i = 0; i < paths.length; i++) {
-				paths[i] = configuredDbBasePaths[i].toString();
+				paths[i] = localRocksDbDirectories[i].toString();
 			}
 			return paths;
 		}
@@ -403,7 +404,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * @param options The options to set (must not be null).
 	 */
 	public void setPredefinedOptions(PredefinedOptions options) {
-		predefinedOptions = requireNonNull(options);
+		predefinedOptions = checkNotNull(options);
 	}
 
 	/**
@@ -496,7 +497,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	public String toString() {
 		return "RocksDB State Backend {" +
 			"isInitialized=" + isInitialized +
-			", configuredDbBasePaths=" + Arrays.toString(configuredDbBasePaths) +
+			", configuredDbBasePaths=" + Arrays.toString(localRocksDbDirectories) +
 			", initializedDbBasePaths=" + Arrays.toString(initializedDbBasePaths) +
 			", checkpointStreamBackend=" + checkpointStreamBackend +
 			'}';


[08/17] flink git commit: [FLINK-5823] [checkpoints] State backends define checkpoint and savepoint directories, improved configuration

Posted by se...@apache.org.
[FLINK-5823] [checkpoints] State backends define checkpoint and savepoint directories, improved configuration


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

Branch: refs/heads/master
Commit: fa03e78d3a245b40ceb3efffeb3020853e74e48b
Parents: 7d820d6
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 25 19:04:10 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../state/RocksDBStateBackendFactory.java       |  25 +-
 .../org/apache/flink/util/TernaryBoolean.java   |  91 +++++
 .../apache/flink/util/TernaryBooleanTest.java   |  79 +++++
 .../runtime/state/AbstractStateBackend.java     | 165 +--------
 .../runtime/state/ConfigurableStateBackend.java |  45 +++
 .../flink/runtime/state/StateBackendLoader.java | 265 +++++++++++++++
 .../filesystem/AbstractFileStateBackend.java    | 206 +++++++++++
 .../state/filesystem/FsStateBackend.java        | 327 ++++++++++++------
 .../state/filesystem/FsStateBackendFactory.java |  35 +-
 .../state/memory/MemoryStateBackend.java        | 253 ++++++++++++--
 .../state/memory/MemoryStateBackendFactory.java |  35 ++
 ...ExecutionGraphCheckpointCoordinatorTest.java |   3 +-
 .../ArchivedExecutionGraphTest.java             |  11 +-
 .../runtime/jobmanager/JobManagerTest.java      |  15 +-
 .../runtime/state/MemoryStateBackendTest.java   |   3 +-
 .../runtime/state/StateBackendLoadingTest.java  | 340 +++++++++++++++++--
 .../flink/streaming/api/graph/StreamConfig.java |   6 +-
 .../streaming/runtime/tasks/StreamTask.java     |  20 +-
 .../tasks/TaskCheckpointingBehaviourTest.java   |  24 +-
 .../PojoSerializerUpgradeTest.java              |  17 +-
 20 files changed, 1569 insertions(+), 396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 f0569b8..de5be9a 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.contrib.streaming.state;
 
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.Path;
@@ -37,33 +38,29 @@ public class RocksDBStateBackendFactory implements StateBackendFactory<RocksDBSt
 
 	protected static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackendFactory.class);
 
-	private static final long serialVersionUID = 4906988360901930371L;
-
-	/** The key under which the config stores the directory where checkpoints should be stored. */
-	public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
-	/** The key under which the config stores the directory where RocksDB should be stored. */
-	public static final String ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.rocksdb.checkpointdir";
-
 	@Override
 	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);
+		final String checkpointDirURI = config.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY);
+		final String rocksdbLocalPaths = config.getString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES);
+		final boolean incrementalCheckpoints = config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS);
 
 		if (checkpointDirURI == null) {
 			throw new IllegalConfigurationException(
 				"Cannot create the RocksDB state backend: The configuration does not specify the " +
-				"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
+				"checkpoint directory '" + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key() + '\'');
 		}
 
 		try {
-			Path path = new Path(checkpointDirURI);
-			RocksDBStateBackend backend = new RocksDBStateBackend(path.toUri());
-			if (rocksdbLocalPath != null) {
-				String[] directories = rocksdbLocalPath.split(",|" + File.pathSeparator);
+			final Path path = new Path(checkpointDirURI);
+			final RocksDBStateBackend backend = new RocksDBStateBackend(path.toUri(), incrementalCheckpoints);
+
+			if (rocksdbLocalPaths != null) {
+				String[] directories = rocksdbLocalPaths.split(",|" + File.pathSeparator);
 				backend.setDbStoragePaths(directories);
 			}
+
 			LOG.info("State backend is set to RocksDB (configured DB storage paths {}, checkpoints to filesystem {} ) ",
 					backend.getDbStoragePaths(), path);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java b/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
new file mode 100644
index 0000000..14ef24b
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.util;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import javax.annotation.Nullable;
+
+/**
+ * A ternary boolean, which can have the values 'true', 'false', or 'undefined'.
+ *
+ * <p>A ternary boolean can for example be used to configuration switches that
+ * may be not configured (undefined), in which case a default value should be assumed.
+ */
+@PublicEvolving
+public enum TernaryBoolean {
+
+	/** The value for 'true'. */
+	TRUE,
+
+	/** The value for 'false'. */
+	FALSE,
+
+	/** The value for 'undefined'. In a configuration setting, this typically means that the
+	 * default value will be used, or the value from a deployment-wide configuration.*/
+	UNDEFINED;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the boolean value corresponding to this value. If this is the 'undefined' value,
+	 * the method returns the given default.
+	 *
+	 * @param defaultValue The value to be returned in case this ternary value is 'undefined'.
+	 */
+	public boolean getOrDefault(boolean defaultValue) {
+		return this == UNDEFINED ? defaultValue : (this == TRUE);
+	}
+
+	/**
+	 * Gets the boolean value corresponding to this value. If this is the 'UNDEFINED' value,
+	 * the method returns the given valueForUndefined.
+	 *
+	 * @param valueForUndefined The value to be returned in case this ternary value is 'undefined'.
+	 */
+	public TernaryBoolean resolveUndefined(boolean valueForUndefined) {
+		return this != UNDEFINED ? this : fromBoolean(valueForUndefined);
+	}
+
+	/**
+	 * Gets this ternary boolean as a boxed boolean. The value 'undefined' results
+	 * in 'null.
+	 */
+	@Nullable
+	public Boolean getAsBoolean() {
+		return this == UNDEFINED ? null : (this == TRUE ? Boolean.TRUE : Boolean.FALSE);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Converts the given boolean to a TernaryBoolean, {@link #TRUE} or {@link #FALSE} respectively.
+	 */
+	public static TernaryBoolean fromBoolean(boolean bool) {
+		return bool ? TRUE : FALSE;
+	}
+
+	/**
+	 * Converts the given boxed Boolean to a TernaryBoolean. A null value results in
+	 * {@link #UNDEFINED}, while a non-null value results in {@link #TRUE} or {@link #FALSE} respectively.
+	 */
+	public static TernaryBoolean fromBoxedBoolean(@Nullable Boolean bool) {
+		return bool == null ? UNDEFINED : fromBoolean(bool);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-core/src/test/java/org/apache/flink/util/TernaryBooleanTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/TernaryBooleanTest.java b/flink-core/src/test/java/org/apache/flink/util/TernaryBooleanTest.java
new file mode 100644
index 0000000..866fafb
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/util/TernaryBooleanTest.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.util;
+
+import org.junit.Test;
+
+import static org.apache.flink.util.TernaryBoolean.FALSE;
+import static org.apache.flink.util.TernaryBoolean.TRUE;
+import static org.apache.flink.util.TernaryBoolean.UNDEFINED;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link TernaryBoolean} class.
+ */
+public class TernaryBooleanTest {
+
+	@Test
+	public void testWithDefault() {
+		assertTrue(TRUE.getOrDefault(true));
+		assertTrue(TRUE.getOrDefault(false));
+
+		assertFalse(FALSE.getOrDefault(true));
+		assertFalse(FALSE.getOrDefault(false));
+
+		assertTrue(UNDEFINED.getOrDefault(true));
+		assertFalse(UNDEFINED.getOrDefault(false));
+	}
+
+	@Test
+	public void testResolveUndefined() {
+		assertEquals(TRUE, TRUE.resolveUndefined(true));
+		assertEquals(TRUE, TRUE.resolveUndefined(false));
+
+		assertEquals(FALSE, FALSE.resolveUndefined(true));
+		assertEquals(FALSE, FALSE.resolveUndefined(false));
+
+		assertEquals(TRUE, UNDEFINED.resolveUndefined(true));
+		assertEquals(FALSE, UNDEFINED.resolveUndefined(false));
+	}
+
+	@Test
+	public void testToBoolean() {
+		assertTrue(Boolean.TRUE == TRUE.getAsBoolean());
+		assertTrue(Boolean.FALSE == FALSE.getAsBoolean());
+		assertNull(UNDEFINED.getAsBoolean());
+	}
+
+	@Test
+	public void testFromBoolean() {
+		assertEquals(TRUE, TernaryBoolean.fromBoolean(true));
+		assertEquals(FALSE, TernaryBoolean.fromBoolean(false));
+	}
+
+	@Test
+	public void testFromBoxedBoolean() {
+		assertEquals(TRUE, TernaryBoolean.fromBoxedBoolean(Boolean.TRUE));
+		assertEquals(FALSE, TernaryBoolean.fromBoxedBoolean(Boolean.FALSE));
+		assertEquals(UNDEFINED, TernaryBoolean.fromBoxedBoolean(null));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 1594e2e..c72f012 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,27 +21,16 @@ 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>
+ *
+ * <p>This class has currently no contents and only kept to not break the prior class hierarchy for users.
  */
 @PublicEvolving
 public abstract class AbstractStateBackend implements StateBackend, java.io.Serializable {
@@ -49,19 +38,6 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri
 	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
 	// ------------------------------------------------------------------------
 
@@ -94,141 +70,4 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri
 	public abstract OperatorStateBackend createOperatorStateBackend(
 			Environment env,
 			String operatorIdentifier) throws Exception;
-
-	// ------------------------------------------------------------------------
-	//  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/fa03e78d/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java
new file mode 100644
index 0000000..f509e8d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+
+/**
+ * An interface for state backends that pick up additional parameters from a configuration.
+ */
+public interface ConfigurableStateBackend {
+
+	/**
+	 * Creates a variant of the state backend that applies additional configuration parameters.
+	 *
+	 * <p>Settings that were directly done on the original state backend object in the application
+	 * program typically have precedence over setting picked up from the configuration.
+	 *
+	 * <p>If no configuration is applied, or if the method directly applies configuration values to
+	 * the (mutable) state backend object, this method may return the original state backend object.
+	 * Otherwise it typically returns a modified copy.
+	 *
+	 * @param config The configuration to pick the values from.
+	 * @return A reconfigured state backend.
+	 *
+	 * @throws IllegalConfigurationException Thrown if the configuration contained invalid entries.
+	 */
+	StateBackend configure(Configuration config) throws IllegalConfigurationException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
new file mode 100644
index 0000000..857dfc1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
@@ -0,0 +1,265 @@
+/*
+ * 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.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+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.state.memory.MemoryStateBackendFactory;
+import org.apache.flink.util.DynamicCodeLoadingException;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class contains utility methods to load state backends from configurations.
+ */
+public class StateBackendLoader {
+
+	// ------------------------------------------------------------------------
+	//  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";
+
+	// ------------------------------------------------------------------------
+	//  Loading the state backend from a configuration 
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Loads the state backend from the configuration, from the parameter 'state.backend', as defined
+	 * in {@link CheckpointingOptions#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 StateBackendLoader#MEMORY_STATE_BACKEND_NAME}',
+	 * '{@value StateBackendLoader#FS_STATE_BACKEND_NAME}', and
+	 * '{@value StateBackendLoader#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(CheckpointingOptions.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:
+				MemoryStateBackend memBackend = new MemoryStateBackendFactory().createFromConfig(config);
+
+				if (logger != null) {
+					Path memExternalized = memBackend.getCheckpointPath();
+					String extern = memExternalized == null ? "" :
+							" (externalized to " + memExternalized + ')';
+					logger.info("State backend is set to heap memory (checkpoint to JobManager) {}", extern);
+				}
+				return memBackend;
+
+			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.getCheckpointPath());
+				}
+				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 '" +
+							CheckpointingOptions.STATE_BACKEND.key() + "' is not a valid state backend factory (" +
+							backendName + ')', e);
+				}
+
+				return factory.createFromConfig(config);
+		}
+	}
+
+	/**
+	 * Checks if an application-defined state backend is given, and if not, loads the state
+	 * backend from the configuration, from the parameter 'state.backend', as defined
+	 * in {@link CheckpointingOptions#STATE_BACKEND}. If no state backend is configured, this instantiates the
+	 * default state backend (the {@link MemoryStateBackend}). 
+	 *
+	 * <p>If an application-defined state backend is found, and the state backend is a
+	 * {@link ConfigurableStateBackend}, this methods calls {@link ConfigurableStateBackend#configure(Configuration)}
+	 * on the state backend.
+	 *
+	 * <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 fromApplicationOrConfigOrDefault(
+			@Nullable StateBackend fromApplication,
+			Configuration config,
+			ClassLoader classLoader,
+			@Nullable Logger logger) throws IllegalConfigurationException, DynamicCodeLoadingException, IOException {
+
+		checkNotNull(config, "config");
+		checkNotNull(classLoader, "classLoader");
+
+		final StateBackend backend;
+
+		// (1) the application defined state backend has precedence
+		if (fromApplication != null) {
+			if (logger != null) {
+				logger.info("Using application-defined state backend: {}", fromApplication);
+			}
+
+			// see if this is supposed to pick up additional configuration parameters
+			if (fromApplication instanceof ConfigurableStateBackend) {
+				// needs to pick up configuration
+				if (logger != null) {
+					logger.info("Configuring application-defined state backend with job/cluster config");
+				}
+
+				backend = ((ConfigurableStateBackend) fromApplication).configure(config);
+			}
+			else {
+				// keep as is!
+				backend = fromApplication;
+			}
+		}
+		else {
+			// (2) check if the config defines a state backend
+			final StateBackend fromConfig = loadStateBackendFromConfig(config, classLoader, logger);
+			if (fromConfig != null) {
+				backend = fromConfig;
+			}
+			else {
+				// (3) use the default
+				backend = new MemoryStateBackendFactory().createFromConfig(config);
+				if (logger != null) {
+					logger.info("No state backend has been configured, using default (Memory / JobManager) {}", backend);
+				}
+			}
+		}
+
+		// to keep supporting the old behavior where default (JobManager) Backend + HA mode = checkpoints in HA store
+		// we add the HA persistence dir as the checkpoint directory if none other is set
+
+		if (backend instanceof MemoryStateBackend) {
+			final MemoryStateBackend memBackend = (MemoryStateBackend) backend;
+
+			if (memBackend.getCheckpointPath() == null && HighAvailabilityMode.isHighAvailabilityModeActivated(config)) {
+				final String haStoragePath = config.getString(HighAvailabilityOptions.HA_STORAGE_PATH);
+
+				if (haStoragePath != null) {
+					try {
+						Path checkpointDirPath = new Path(haStoragePath, UUID.randomUUID().toString());
+						if (checkpointDirPath.toUri().getScheme() == null) {
+							checkpointDirPath = checkpointDirPath.makeQualified(checkpointDirPath.getFileSystem());
+						}
+						Configuration tempConfig = new Configuration(config);
+						tempConfig.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDirPath.toString());
+						return memBackend.configure(tempConfig);
+					} catch (Exception ignored) {}
+				}
+			}
+		}
+
+		return backend;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated */
+	private StateBackendLoader() {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
new file mode 100644
index 0000000..6ec6f24
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
@@ -0,0 +1,206 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.ConfigOption;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.net.URI;
+
+/**
+ * A base class for all state backends that store their metadata (and data) in files.
+ * Examples that inherit from this are the {@link FsStateBackend}, the
+ * {@link org.apache.flink.runtime.state.memory.MemoryStateBackend MemoryStateBackend}, or the
+ * {@code RocksDBStateBackend}.
+ *
+ * <p>This class takes the base checkpoint- and savepoint directory paths, but also accepts null
+ * for both of then, in which case creating externalized checkpoint is not possible, and it is not
+ * possible to create a savepoint with a default path. Null is accepted to enable implementations
+ * that only optionally support default savepoints and externalized checkpoints.
+ *
+ * <h1>Checkpoint Layout</h1>
+ *
+ * The state backend is configured with a base directory and persists the checkpoint data of specific
+ * checkpoints in specific subdirectories. For example, if the base directory was set to 
+ * {@code hdfs://namenode:port/flink-checkpoints/}, the state backend will create a subdirectory with
+ * the job's ID that will contain the actual checkpoints:
+ * ({@code hdfs://namenode:port/flink-checkpoints/1b080b6e710aabbef8993ab18c6de98b})
+ *
+ * <p>Each checkpoint individually will store all its files in a subdirectory that includes the
+ * checkpoint number, such as {@code hdfs://namenode:port/flink-checkpoints/1b080b6e710aabbef8993ab18c6de98b/chk-17/}.
+ *
+ * <h1>Savepoint Layout</h1>
+ *
+ * A savepoint that is set to be stored in path {@code hdfs://namenode:port/flink-savepoints/}, will create
+ * a subdirectory {@code savepoint-jobId(0, 6)-randomDigits} in which it stores all savepoint data.
+ * The random digits are added as "entropy" to avoid directory collisions.
+ */
+@PublicEvolving
+public abstract class AbstractFileStateBackend extends AbstractStateBackend {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(AbstractFileStateBackend.class);
+
+	// ------------------------------------------------------------------------
+	//  State Backend Properties
+	// ------------------------------------------------------------------------
+
+	/** The path where checkpoints will be stored, or null, if none has been configured. */
+	@Nullable
+	private final Path baseCheckpointPath;
+
+	/** The path where savepoints will be stored, or null, if none has been configured. */
+	@Nullable
+	private final Path baseSavepointPath;
+
+	/**
+	 * Creates a backend with the given optional checkpoint- and savepoint base directories.
+	 *
+	 * @param baseCheckpointPath The base directory for checkpoints, or null, if none is configured.
+	 * @param baseSavepointPath The default directory for savepoints, or null, if none is set.
+	 */
+	protected AbstractFileStateBackend(
+			@Nullable URI baseCheckpointPath,
+			@Nullable URI baseSavepointPath) {
+
+		this(baseCheckpointPath == null ? null : new Path(baseCheckpointPath),
+				baseSavepointPath == null ? null : new Path(baseSavepointPath));
+	}
+
+	/**
+	 * Creates a backend with the given optional checkpoint- and savepoint base directories.
+	 *
+	 * @param baseCheckpointPath The base directory for checkpoints, or null, if none is configured.
+	 * @param baseSavepointPath The default directory for savepoints, or null, if none is set.
+	 */
+	protected AbstractFileStateBackend(
+			@Nullable Path baseCheckpointPath,
+			@Nullable Path baseSavepointPath) {
+
+		this.baseCheckpointPath = baseCheckpointPath == null ? null : validatePath(baseCheckpointPath);
+		this.baseSavepointPath = baseSavepointPath == null ? null : validatePath(baseSavepointPath);
+	}
+
+	/**
+	 * Creates a new backend using the given checkpoint-/savepoint directories, or the values defined in
+	 * the given configuration. If a checkpoint-/savepoint parameter is not null, that value takes precedence
+	 * over the value in the configuration. If the configuration does not specify a value, it is possible
+	 * that the checkpoint-/savepoint directories in the backend will be null.
+	 *
+	 * <p>This constructor can be used to create a backend that is based partially on a given backend
+	 * and partially on a configuration.
+	 *
+	 * @param baseCheckpointPath The checkpoint base directory to use (or null).
+	 * @param baseSavepointPath The default savepoint directory to use (or null).
+	 * @param configuration The configuration to read values from 
+	 */
+	protected AbstractFileStateBackend(
+			@Nullable Path baseCheckpointPath,
+			@Nullable Path baseSavepointPath,
+			Configuration configuration) {
+
+		this(parameterOrConfigured(baseCheckpointPath, configuration, CheckpointingOptions.CHECKPOINTS_DIRECTORY),
+				parameterOrConfigured(baseSavepointPath, configuration, CheckpointingOptions.SAVEPOINT_DIRECTORY));
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the checkpoint base directory. Jobs will create job-specific subdirectories
+	 * for checkpoints within this directory. May be null, if not configured.
+	 *
+	 * @return The checkpoint base directory
+	 */
+	@Nullable
+	public Path getCheckpointPath() {
+		return baseCheckpointPath;
+	}
+
+	/**
+	 * Gets the directory where savepoints are stored by default (when no custom path is given
+	 * to the savepoint trigger command).
+	 *
+	 * @return The default directory for savepoints, or null, if no default directory has been configured.
+	 */
+	@Nullable
+	public Path getSavepointPath() {
+		return baseSavepointPath;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	// 
+	/**
+	 * Checks the validity of the path's scheme and path.
+	 *
+	 * @param path The path to check.
+	 * @return The URI as a Path.
+	 *
+	 * @throws IllegalArgumentException Thrown, if the URI misses scheme or path.
+	 */
+	private static Path validatePath(Path path) {
+		final URI uri = path.toUri();
+		final String scheme = uri.getScheme();
+		final String pathPart = uri.getPath();
+
+		// some validity checks
+		if (scheme == null) {
+			throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " +
+					"Please specify the file system scheme explicitly in the URI.");
+		}
+		if (pathPart == null) {
+			throw new IllegalArgumentException("The path to store the checkpoint data in is null. " +
+					"Please specify a directory path for the checkpoint data.");
+		}
+		if (pathPart.length() == 0 || pathPart.equals("/")) {
+			throw new IllegalArgumentException("Cannot use the root directory for checkpoints.");
+		}
+
+		return path;
+	}
+
+	@Nullable
+	private static Path parameterOrConfigured(@Nullable Path path, Configuration config, ConfigOption<String> option) {
+		if (path != null) {
+			return path;
+		}
+		else {
+			String configValue = config.getString(option);
+			try {
+				return configValue == null ? null : new Path(configValue);
+			}
+			catch (IllegalArgumentException e) {
+				throw new IllegalConfigurationException("Cannot parse value for " + option.key() +
+						" : " + configValue + " . Not a valid path.");
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 952988f..2fff45a 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
@@ -18,54 +18,91 @@
 
 package org.apache.flink.runtime.state.filesystem;
 
+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.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.util.TernaryBoolean;
+
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 import java.io.IOException;
 import java.net.URI;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * The file state backend is a state backend that stores the state of streaming jobs in a file system.
+ * This state backend holds the working state in the memory (JVM heap) of the TaskManagers.
+ * The state backend checkpoints state as files to a file system (hence the backend's name).
+ *
+ * <p>Each checkpoint individually will store all its files in a subdirectory that includes the
+ * checkpoint number, such as {@code hdfs://namenode:port/flink-checkpoints/chk-17/}.
+ *
+ * <h1>State Size Considerations</h1>
+ *
+ * <p>Working state is kept on the TaskManager heap. If a TaskManager executes multiple
+ * tasks concurrently (if the TaskManager has multiple slots, or if slot-sharing is used)
+ * then the aggregate state of all tasks needs to fit into that TaskManager's memory.
+ *
+ * <p>This state backend stores small state chunks directly with the metadata, to avoid creating
+ * many small files. The threshold for that is configurable. When increasing this threshold, the
+ * size of the checkpoint metadata increases. The checkpoint metadata of all retained completed
+ * checkpoints needs to fit into the JobManager's heap memory. This is typically not a problem,
+ * unless the threshold {@link #getMinFileSizeThreshold()} is increased significantly.
+ *
+ * <h1>Persistence Guarantees</h1>
  *
- * <p>The state backend has one core directory into which it puts all checkpoint data. Inside that
- * directory, it creates a directory per job, inside which each checkpoint gets a directory, with
- * files for each state, for example:
+ * <p>Checkpoints from this state backend are as persistent and available as filesystem that is written to.
+ * If the file system is a persistent distributed file system, this state backend supports
+ * highly available setups. The backend additionally supports savepoints and externalized checkpoints.
  *
- * {@code hdfs://namenode:port/flink-checkpoints/<job-id>/chk-17/6ba7b810-9dad-11d1-80b4-00c04fd430c8 }
+ * <h1>Configuration</h1>
+ *
+ * <p>As for all state backends, this backend can either be configured within the application (by creating
+ * the backend with the respective constructor parameters and setting it on the execution environment)
+ * or by specifying it in the Flink configuration.
+ *
+ * <p>If the state backend was specified in the application, it may pick up additional configuration
+ * parameters from the Flink configuration. For example, if the backend if configured in the application
+ * without a default savepoint directory, it will pick up a default savepoint directory specified in the
+ * Flink configuration of the running job/cluster. That behavior is implemented via the
+ * {@link #configure(Configuration)} method.
  */
-public class FsStateBackend extends AbstractStateBackend {
+@PublicEvolving
+public class FsStateBackend extends AbstractFileStateBackend implements ConfigurableStateBackend {
 
 	private static final long serialVersionUID = -8191916350224044011L;
 
-	/** By default, state smaller than 1024 bytes will not be written to files, but
-	 * will be stored directly with the metadata */
-	public static final int DEFAULT_FILE_STATE_THRESHOLD = 1024;
+	/** Maximum size of state that is stored with the metadata, rather than in files (1 MiByte). */
+	public static final int MAX_FILE_STATE_THRESHOLD = 1024 * 1024;
 
-	/** Maximum size of state that is stored with the metadata, rather than in files */
-	private static final int MAX_FILE_STATE_THRESHOLD = 1024 * 1024;
-	
-	/** The path to the directory for the checkpoint data, including the file system
-	 * description via scheme and optional authority */
-	private final Path basePath;
+	// ------------------------------------------------------------------------
 
-	/** State below this size will be stored as part of the metadata, rather than in files */
+	/** State below this size will be stored as part of the metadata, rather than in files.
+	 * A value of '-1' means not yet configured, in which case the default will be used. */
 	private final int fileStateThreshold;
 
-	/** Switch to chose between synchronous and asynchronous snapshots */
-	private final boolean asynchronousSnapshots;
+	/** Switch to chose between synchronous and asynchronous snapshots.
+	 * A value of 'undefined' means not yet configured, in which case the default will be used. */
+	private final TernaryBoolean asynchronousSnapshots;
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Creates a new state backend that stores its checkpoint data in the file system and location
@@ -80,9 +117,8 @@ public class FsStateBackend extends AbstractStateBackend {
 	 *
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(String checkpointDataUri) throws IOException {
+	public FsStateBackend(String checkpointDataUri) {
 		this(new Path(checkpointDataUri));
 	}
 
@@ -100,10 +136,8 @@ public class FsStateBackend extends AbstractStateBackend {
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
 	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
-	 *
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(String checkpointDataUri, boolean asynchronousSnapshots) throws IOException {
+	public FsStateBackend(String checkpointDataUri, boolean asynchronousSnapshots) {
 		this(new Path(checkpointDataUri), asynchronousSnapshots);
 	}
 
@@ -120,9 +154,8 @@ public class FsStateBackend extends AbstractStateBackend {
 	 *
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(Path checkpointDataUri) throws IOException {
+	public FsStateBackend(Path checkpointDataUri) {
 		this(checkpointDataUri.toUri());
 	}
 
@@ -140,10 +173,8 @@ public class FsStateBackend extends AbstractStateBackend {
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
 	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
-	 *
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(Path checkpointDataUri, boolean asynchronousSnapshots) throws IOException {
+	public FsStateBackend(Path checkpointDataUri, boolean asynchronousSnapshots) {
 		this(checkpointDataUri.toUri(), asynchronousSnapshots);
 	}
 
@@ -160,10 +191,30 @@ public class FsStateBackend extends AbstractStateBackend {
 	 *
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(URI checkpointDataUri) throws IOException {
-		this(checkpointDataUri, DEFAULT_FILE_STATE_THRESHOLD, true);
+	public FsStateBackend(URI checkpointDataUri) {
+		this(checkpointDataUri, null, -1, TernaryBoolean.UNDEFINED);
+	}
+
+	/**
+	 * Creates a new state backend that stores its checkpoint data in the file system and location
+	 * defined by the given URI. Optionally, this constructor accepts a default savepoint storage
+	 * directory to which savepoints are stored when no custom target path is give to the savepoint
+	 * command.
+	 *
+	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+	 * must be accessible via {@link FileSystem#get(URI)}.
+	 *
+	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+	 * (host and port), or that the Hadoop configuration that describes that information must be in the
+	 * classpath.
+	 *
+	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+	 *                          and the path to the checkpoint data directory.
+	 * @param defaultSavepointDirectory The default directory to store savepoints to. May be null.
+	 */
+	public FsStateBackend(URI checkpointDataUri, @Nullable URI defaultSavepointDirectory) {
+		this(checkpointDataUri, defaultSavepointDirectory, -1, TernaryBoolean.UNDEFINED);
 	}
 
 	/**
@@ -180,11 +231,10 @@ public class FsStateBackend extends AbstractStateBackend {
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
 	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
-	 *
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
-	public FsStateBackend(URI checkpointDataUri, boolean asynchronousSnapshots) throws IOException {
-		this(checkpointDataUri, DEFAULT_FILE_STATE_THRESHOLD, asynchronousSnapshots);
+	public FsStateBackend(URI checkpointDataUri, boolean asynchronousSnapshots) {
+		this(checkpointDataUri, null, -1,
+				TernaryBoolean.fromBoolean(asynchronousSnapshots));
 	}
 
 	/**
@@ -202,13 +252,9 @@ public class FsStateBackend extends AbstractStateBackend {
 	 *                          and the path to the checkpoint data directory.
 	 * @param fileStateSizeThreshold State up to this size will be stored as part of the metadata,
 	 *                             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 {
-
-		this(checkpointDataUri, fileStateSizeThreshold, true);
+	public FsStateBackend(URI checkpointDataUri, int fileStateSizeThreshold) {
+		this(checkpointDataUri, null, fileStateSizeThreshold, TernaryBoolean.UNDEFINED);
 	}
 
 	/**
@@ -225,34 +271,120 @@ public class FsStateBackend extends AbstractStateBackend {
 	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
 	 *                          and the path to the checkpoint data directory.
 	 * @param fileStateSizeThreshold State up to this size will be stored as part of the metadata,
-	 *                             rather than in files
+	 *                             rather than in files (-1 for default value).
 	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
-	 *
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
 	public FsStateBackend(
 			URI checkpointDataUri,
 			int fileStateSizeThreshold,
-			boolean asynchronousSnapshots) throws IOException {
+			boolean asynchronousSnapshots) {
 
-		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(checkpointDataUri, null, fileStateSizeThreshold,
+				TernaryBoolean.fromBoolean(asynchronousSnapshots));
+	}
 
-		this.fileStateThreshold = fileStateSizeThreshold;
-		this.basePath = validateAndNormalizeUri(checkpointDataUri);
+	/**
+	 * Creates a new state backend that stores its checkpoint data in the file system and location
+	 * defined by the given URI.
+	 *
+	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+	 * must be accessible via {@link FileSystem#get(URI)}.
+	 *
+	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+	 * (host and port), or that the Hadoop configuration that describes that information must be in the
+	 * classpath.
+	 *
+	 * @param checkpointDirectory        The path to write checkpoint metadata to.
+	 * @param defaultSavepointDirectory  The path to write savepoints to. If null, the value from
+	 *                                   the runtime configuration will be used, or savepoint
+	 *                                   target locations need to be passed when triggering a savepoint.
+	 * @param fileStateSizeThreshold     State below this size will be stored as part of the metadata,
+	 *                                   rather than in files. If -1, the value configured in the
+	 *                                   runtime configuration will be used, or the default value (1KB)
+	 *                                   if nothing is configured.
+	 * @param asynchronousSnapshots      Flag to switch between synchronous and asynchronous
+	 *                                   snapshot mode. If UNDEFINED, the value configured in the
+	 *                                   runtime configuration will be used.
+	 */
+	public FsStateBackend(
+			URI checkpointDirectory,
+			@Nullable URI defaultSavepointDirectory,
+			int fileStateSizeThreshold,
+			TernaryBoolean asynchronousSnapshots) {
+
+		super(checkNotNull(checkpointDirectory, "checkpoint directory is null"), defaultSavepointDirectory);
 
+		checkNotNull(asynchronousSnapshots, "asynchronousSnapshots");
+		checkArgument(fileStateSizeThreshold >= -1 && fileStateSizeThreshold <= MAX_FILE_STATE_THRESHOLD,
+				"The threshold for file state size must be in [-1, %s], where '-1' means to use " +
+						"the value from the deployment's configuration.", MAX_FILE_STATE_THRESHOLD);
+
+		this.fileStateThreshold = fileStateSizeThreshold;
 		this.asynchronousSnapshots = asynchronousSnapshots;
 	}
 
 	/**
-	 * Gets the base directory where all state-containing files are stored.
-	 * The job specific directory is created inside this directory.
+	 * Private constructor that creates a re-configured copy of the state backend.
 	 *
-	 * @return The base directory.
+	 * @param original The state backend to re-configure
+	 * @param configuration The configuration
 	 */
+	private FsStateBackend(FsStateBackend original, Configuration configuration) {
+		super(original.getCheckpointPath(), original.getSavepointPath(), configuration);
+
+		// if asynchronous snapshots were configured, use that setting,
+		// else check the configuration
+		this.asynchronousSnapshots = original.asynchronousSnapshots.resolveUndefined(
+				configuration.getBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS));
+
+		final int sizeThreshold = original.fileStateThreshold >= 0 ?
+				original.fileStateThreshold :
+				configuration.getInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD);
+
+		if (sizeThreshold >= 0 && sizeThreshold <= MAX_FILE_STATE_THRESHOLD) {
+			this.fileStateThreshold = sizeThreshold;
+		}
+		else {
+			this.fileStateThreshold = CheckpointingOptions.FS_SMALL_FILE_THRESHOLD.defaultValue();
+
+			// because this is the only place we (unlikely) ever log, we lazily
+			// create the logger here
+			LoggerFactory.getLogger(AbstractFileStateBackend.class).warn(
+					"Ignoring invalid file size threshold value ({}): {} - using default value {} instead.",
+					CheckpointingOptions.FS_SMALL_FILE_THRESHOLD.key(), sizeThreshold,
+					CheckpointingOptions.FS_SMALL_FILE_THRESHOLD.defaultValue());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the base directory where all the checkpoints are stored.
+	 * The job-specific checkpoint directory is created inside this directory.
+	 *
+	 * @return The base directory for checkpoints.
+	 *
+	 * @deprecated Deprecated in favor of {@link #getCheckpointPath()}.
+	 */
+	@Deprecated
 	public Path getBasePath() {
-		return basePath;
+		return getCheckpointPath();
+	}
+
+	/**
+	 * Gets the base directory where all the checkpoints are stored.
+	 * The job-specific checkpoint directory is created inside this directory.
+	 *
+	 * @return The base directory for checkpoints.
+	 */
+	@Nonnull
+	@Override
+	public Path getCheckpointPath() {
+		// we know that this can never be null by the way of constructor checks
+		//noinspection ConstantConditions
+		return super.getCheckpointPath();
 	}
 
 	/**
@@ -260,12 +392,41 @@ public class FsStateBackend extends AbstractStateBackend {
 	 * 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}.
+	 * <p>If not explicitly configured, this is the default value of
+	 * {@link CheckpointingOptions#FS_SMALL_FILE_THRESHOLD}.
 	 *
 	 * @return The file size threshold, in bytes.
 	 */
 	public int getMinFileSizeThreshold() {
-		return fileStateThreshold;
+		return fileStateThreshold >= 0 ?
+				fileStateThreshold :
+				CheckpointingOptions.FS_SMALL_FILE_THRESHOLD.defaultValue();
+	}
+
+	/**
+	 * Gets whether the key/value data structures are asynchronously snapshotted.
+	 *
+	 * <p>If not explicitly configured, this is the default value of
+	 * {@link CheckpointingOptions#ASYNC_SNAPSHOTS}.
+	 */
+	public boolean isUsingAsynchronousSnapshots() {
+		return asynchronousSnapshots.getOrDefault(CheckpointingOptions.ASYNC_SNAPSHOTS.defaultValue());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reconfiguration
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a copy of this state backend that uses the values defined in the configuration
+	 * for fields where that were not specified in this state backend.
+	 *
+	 * @param config the configuration
+	 * @return The re-configured variant of the state backend
+	 */
+	@Override
+	public FsStateBackend configure(Configuration config) {
+		return new FsStateBackend(this, config);
 	}
 
 	// ------------------------------------------------------------------------
@@ -274,7 +435,7 @@ public class FsStateBackend extends AbstractStateBackend {
 
 	@Override
 	public CheckpointStreamFactory createStreamFactory(JobID jobId, String operatorIdentifier) throws IOException {
-		return new FsCheckpointStreamFactory(basePath, jobId, fileStateThreshold);
+		return new FsCheckpointStreamFactory(getCheckpointPath(), jobId, getMinFileSizeThreshold());
 	}
 
 	@Override
@@ -283,7 +444,7 @@ public class FsStateBackend extends AbstractStateBackend {
 			String operatorIdentifier,
 			String targetLocation) throws IOException {
 
-		return new FsSavepointStreamFactory(new Path(targetLocation), jobId, fileStateThreshold);
+		return new FsSavepointStreamFactory(new Path(targetLocation), jobId, getMinFileSizeThreshold());
 	}
 
 	@Override
@@ -302,7 +463,7 @@ public class FsStateBackend extends AbstractStateBackend {
 				env.getUserClassLoader(),
 				numberOfKeyGroups,
 				keyGroupRange,
-				asynchronousSnapshots,
+				isUsingAsynchronousSnapshots(),
 				env.getExecutionConfig());
 	}
 
@@ -314,45 +475,19 @@ public class FsStateBackend extends AbstractStateBackend {
 		return new DefaultOperatorStateBackend(
 			env.getUserClassLoader(),
 			env.getExecutionConfig(),
-			asynchronousSnapshots);
+				isUsingAsynchronousSnapshots());
 	}
 
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
 	@Override
 	public String toString() {
-		return "File State Backend @ " + basePath;
-	}
-
-	/**
-	 * Checks and normalizes the checkpoint data URI. This method first checks the validity of the
-	 * URI (scheme, path, availability of a matching file system) and then normalizes the URI
-	 * to a path.
-	 * 
-	 * <p>If the URI does not include an authority, but the file system configured for the URI has an
-	 * authority, then the normalized path will include this authority.
-	 * 
-	 * @param checkpointDataUri The URI to check and normalize.
-	 * @return A normalized URI as a Path.
-	 * 
-	 * @throws IllegalArgumentException Thrown, if the URI misses scheme or path. 
-	 * @throws IOException Thrown, if no file system can be found for the URI's scheme.
-	 */
-	private static Path validateAndNormalizeUri(URI checkpointDataUri) throws IOException {
-		final String scheme = checkpointDataUri.getScheme();
-		final String path = checkpointDataUri.getPath();
-
-		// some validity checks
-		if (scheme == null) {
-			throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " +
-					"Please specify the file system scheme explicitly in the URI.");
-		}
-		if (path == null) {
-			throw new IllegalArgumentException("The path to store the checkpoint data in is null. " +
-					"Please specify a directory path for the checkpoint data.");
-		}
-		if (path.length() == 0 || path.equals("/")) {
-			throw new IllegalArgumentException("Cannot use the root directory for checkpoints.");
-		}
-
-		return new Path(checkpointDataUri);
+		return "File State Backend (" +
+				"checkpoints: '" + getCheckpointPath() +
+				"', savepoints: '" + getSavepointPath() +
+				"', asynchronous: " + asynchronousSnapshots +
+				", fileStateThreshold: " + fileStateThreshold + ")";
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 4c933ef..2640683 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
@@ -18,45 +18,34 @@
 
 package org.apache.flink.runtime.state.filesystem;
 
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.CheckpointingOptions;
 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.StateBackendFactory;
 
-import java.io.IOException;
-
 /**
- * A factory that creates an {@link org.apache.flink.runtime.state.filesystem.FsStateBackend}
- * from a configuration.
+ * A factory that creates an {@link FsStateBackend} from a configuration.
  */
+@PublicEvolving
 public class FsStateBackendFactory implements StateBackendFactory<FsStateBackend> {
-	
-	/** The key under which the config stores the directory where checkpoints should be stored */
-	public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
-
-	/** 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 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) {
+		// we need to explicitly read the checkpoint directory here, because that
+		// is a required constructor parameter
+		final String checkpointDir = config.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY);
+		if (checkpointDir == null) {
 			throw new IllegalConfigurationException(
 					"Cannot create the file system state backend: The configuration does not specify the " +
-							"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
+							"checkpoint directory '" + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key() + '\'');
 		}
 
 		try {
-			Path path = new Path(checkpointDirURI);
-			return new FsStateBackend(path.toUri(), memoryThreshold);
+			return new FsStateBackend(checkpointDir).configure(config);
 		}
-		catch (IOException | IllegalArgumentException e) {
+		catch (IllegalArgumentException e) {
 			throw new IllegalConfigurationException("Invalid configuration for the state backend", e);
 		}
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/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 b8ebedf..2079a97 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
@@ -18,78 +18,265 @@
 
 package org.apache.flink.runtime.state.memory;
 
+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.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.filesystem.AbstractFileStateBackend;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.util.TernaryBoolean;
+
+import javax.annotation.Nullable;
 
 import java.io.IOException;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+
 /**
- * A {@link AbstractStateBackend} that stores all its data and checkpoints in memory and has no
- * capabilities to spill to disk. Checkpoints are serialized and the serialized data is
- * transferred
+ * This state backend holds the working state in the memory (JVM heap) of the TaskManagers.
+ * The state backend checkpoints state directly to the JobManager's memory (hence the backend's name),
+ * but the checkpoints will be persisted to a file system for high-availability setups and savepoints.
+ * The MemoryStateBackend is consequently a FileSystem-based backend that can work without a
+ * file system dependency in simple setups.
+ *
+ * <p>This state backend should be used only for experimentation, quick local setups,
+ * or for streaming applications that have very small state: Because it requires checkpoints to
+ * go through the JobManager's memory, larger state will occupy larger portions of the JobManager's
+ * main memory, reducing operational stability.
+ * For any other setup, the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend FsStateBackend}
+ * should be used. The {@code FsStateBackend} holds the working state on the TaskManagers in the same way, but
+ * checkpoints state directly to files rather then to the JobManager's memory, thus supporting
+ * large state sizes.
+ *
+ * <h1>State Size Considerations</h1>
+ *
+ * <p>State checkpointing with this state backend is subject to the following conditions:
+ * <ul>
+ *     <li>Each individual state must not exceed the configured maximum state size
+ *         (see {@link #getMaxStateSize()}.</li>
+ *
+ *     <li>All state from one task (i.e., the sum of all operator states and keyed states from all
+ *         chained operators of the task) must not exceed what the RPC system supports, which is
+ *         be default < 10 MB. That limit can be configured up, but that is typically not advised.</li>
+ *
+ *     <li>The sum of all states in the application times all retained checkpoints must comfortably
+ *         fit into the JobManager's JVM heap space.</li>
+ * </ul>
+ *
+ * <h1>Persistence Guarantees</h1>
+ *
+ * <p>For the use cases where the state sizes can be handled by this backend, the backend does guarantee
+ * persistence for savepoints, externalized checkpoints (of configured), and checkpoints
+ * (when high-availability is configured).
+ *
+ * <h1>Configuration</h1>
+ *
+ * <p>As for all state backends, this backend can either be configured within the application (by creating
+ * the backend with the respective constructor parameters and setting it on the execution environment)
+ * or by specifying it in the Flink configuration.
+ *
+ * <p>If the state backend was specified in the application, it may pick up additional configuration
+ * parameters from the Flink configuration. For example, if the backend if configured in the application
+ * without a default savepoint directory, it will pick up a default savepoint directory specified in the
+ * Flink configuration of the running job/cluster. That behavior is implemented via the
+ * {@link #configure(Configuration)} method.
  */
-public class MemoryStateBackend extends AbstractStateBackend {
+@PublicEvolving
+public class MemoryStateBackend extends AbstractFileStateBackend implements ConfigurableStateBackend {
 
 	private static final long serialVersionUID = 4109305377809414635L;
 
-	/** The default maximal size that the snapshotted memory state may have (5 MiBytes) */
-	private static final int DEFAULT_MAX_STATE_SIZE = 5 * 1024 * 1024;
+	/** The default maximal size that the snapshotted memory state may have (5 MiBytes). */
+	public static final int DEFAULT_MAX_STATE_SIZE = 5 * 1024 * 1024;
 
-	/** The maximal size that the snapshotted memory state may have */
+	/** The maximal size that the snapshotted memory state may have. */
 	private final int maxStateSize;
 
-	/** Switch to chose between synchronous and asynchronous snapshots */
-	private final boolean asynchronousSnapshots;
+	/** Switch to chose between synchronous and asynchronous snapshots.
+	 * A value of 'UNDEFINED' means not yet configured, in which case the default will be used. */
+	private final TernaryBoolean asynchronousSnapshots;
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Creates a new memory state backend that accepts states whose serialized forms are
 	 * up to the default state size (5 MB).
+	 *
+	 * <p>Checkpoint and default savepoint locations are used as specified in the
+	 * runtime configuration.
 	 */
 	public MemoryStateBackend() {
-		this(DEFAULT_MAX_STATE_SIZE);
+		this(null, null, DEFAULT_MAX_STATE_SIZE, TernaryBoolean.UNDEFINED);
 	}
 
 	/**
 	 * Creates a new memory state backend that accepts states whose serialized forms are
-	 * up to the given number of bytes.
+	 * up to the default state size (5 MB). The state backend uses asynchronous snapshots
+	 * or synchronous snapshots as configured.
 	 *
-	 * @param maxStateSize The maximal size of the serialized state
+	 * <p>Checkpoint and default savepoint locations are used as specified in the
+	 * runtime configuration.
+	 *
+	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
 	 */
-	public MemoryStateBackend(int maxStateSize) {
-		this(maxStateSize, true);
+	public MemoryStateBackend(boolean asynchronousSnapshots) {
+		this(null, null, DEFAULT_MAX_STATE_SIZE, TernaryBoolean.fromBoolean(asynchronousSnapshots));
 	}
 
 	/**
 	 * Creates a new memory state backend that accepts states whose serialized forms are
-	 * up to the default state size (5 MB).
+	 * up to the given number of bytes.
 	 *
-	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
+	 * <p>Checkpoint and default savepoint locations are used as specified in the
+	 * runtime configuration.
+	 *
+	 * <p><b>WARNING:</b> Increasing the size of this value beyond the default value
+	 * ({@value #DEFAULT_MAX_STATE_SIZE}) should be done with care.
+	 * The checkpointed state needs to be send to the JobManager via limited size RPC messages, and there
+	 * and the JobManager needs to be able to hold all aggregated state in its memory.
+	 *
+	 * @param maxStateSize The maximal size of the serialized state
 	 */
-	public MemoryStateBackend(boolean asynchronousSnapshots) {
-		this(DEFAULT_MAX_STATE_SIZE, asynchronousSnapshots);
+	public MemoryStateBackend(int maxStateSize) {
+		this(null, null, maxStateSize, TernaryBoolean.UNDEFINED);
 	}
 
 	/**
 	 * Creates a new memory state backend that accepts states whose serialized forms are
-	 * up to the given number of bytes.
+	 * up to the given number of bytes and that uses asynchronous snashots as configured.
+	 *
+	 * <p>Checkpoint and default savepoint locations are used as specified in the
+	 * runtime configuration.
+	 *
+	 * <p><b>WARNING:</b> Increasing the size of this value beyond the default value
+	 * ({@value #DEFAULT_MAX_STATE_SIZE}) should be done with care.
+	 * The checkpointed state needs to be send to the JobManager via limited size RPC messages, and there
+	 * and the JobManager needs to be able to hold all aggregated state in its memory.
 	 *
 	 * @param maxStateSize The maximal size of the serialized state
 	 * @param asynchronousSnapshots Switch to enable asynchronous snapshots.
 	 */
 	public MemoryStateBackend(int maxStateSize, boolean asynchronousSnapshots) {
+		this(null, null, maxStateSize, TernaryBoolean.fromBoolean(asynchronousSnapshots));
+	}
+
+	/**
+	 * Creates a new MemoryStateBackend, setting optionally the path to persist checkpoint metadata
+	 * to, and to persist savepoints to.
+	 *
+	 * @param checkpointPath The path to write checkpoint metadata to. If null, the value from
+	 *                       the runtime configuration will be used.
+	 * @param savepointPath  The path to write savepoints to. If null, the value from
+	 *                       the runtime configuration will be used.
+	 */
+	public MemoryStateBackend(@Nullable String checkpointPath, @Nullable String savepointPath) {
+		this(checkpointPath, savepointPath, DEFAULT_MAX_STATE_SIZE, TernaryBoolean.UNDEFINED);
+	}
+
+	/**
+	 * Creates a new MemoryStateBackend, setting optionally the paths to persist checkpoint metadata
+	 * and savepoints to, as well as configuring state thresholds and asynchronous operations.
+	 *
+	 * <p><b>WARNING:</b> Increasing the size of this value beyond the default value
+	 * ({@value #DEFAULT_MAX_STATE_SIZE}) should be done with care.
+	 * The checkpointed state needs to be send to the JobManager via limited size RPC messages, and there
+	 * and the JobManager needs to be able to hold all aggregated state in its memory.
+	 *
+	 * @param checkpointPath The path to write checkpoint metadata to. If null, the value from
+	 *                       the runtime configuration will be used.
+	 * @param savepointPath  The path to write savepoints to. If null, the value from
+	 *                       the runtime configuration will be used.
+	 * @param maxStateSize   The maximal size of the serialized state.
+	 * @param asynchronousSnapshots Flag to switch between synchronous and asynchronous
+	 *                              snapshot mode. If null, the value configured in the
+	 *                              runtime configuration will be used.
+	 */
+	public MemoryStateBackend(
+			@Nullable String checkpointPath,
+			@Nullable String savepointPath,
+			int maxStateSize,
+			TernaryBoolean asynchronousSnapshots) {
+
+		super(checkpointPath == null ? null : new Path(checkpointPath),
+				savepointPath == null ? null : new Path(savepointPath));
+
+		checkArgument(maxStateSize > 0, "maxStateSize must be > 0");
 		this.maxStateSize = maxStateSize;
+
 		this.asynchronousSnapshots = asynchronousSnapshots;
 	}
 
+	/**
+	 * Private constructor that creates a re-configured copy of the state backend.
+	 *
+	 * @param original The state backend to re-configure
+	 * @param configuration The configuration
+	 */
+	private MemoryStateBackend(MemoryStateBackend original, Configuration configuration) {
+		super(original.getCheckpointPath(), original.getSavepointPath(), configuration);
+
+		this.maxStateSize = original.maxStateSize;
+
+		// if asynchronous snapshots were configured, use that setting,
+		// else check the configuration
+		this.asynchronousSnapshots = original.asynchronousSnapshots.resolveUndefined(
+				configuration.getBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS));
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the maximum size that an individual state can have, as configured in the
+	 * constructor (by default {@value #DEFAULT_MAX_STATE_SIZE}).
+	 *
+	 * @return The maximum size that an individual state can have
+	 */
+	public int getMaxStateSize() {
+		return maxStateSize;
+	}
+
+	/**
+	 * Gets whether the key/value data structures are asynchronously snapshotted.
+	 *
+	 * <p>If not explicitly configured, this is the default value of
+	 * {@link CheckpointingOptions#ASYNC_SNAPSHOTS}.
+	 */
+	public boolean isUsingAsynchronousSnapshots() {
+		return asynchronousSnapshots.getOrDefault(CheckpointingOptions.ASYNC_SNAPSHOTS.defaultValue());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reconfiguration
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a copy of this state backend that uses the values defined in the configuration
+	 * for fields where that were not specified in this state backend.
+	 *
+	 * @param config the configuration
+	 * @return The re-configured variant of the state backend
+	 */
+	@Override
+	public MemoryStateBackend configure(Configuration config) {
+		return new MemoryStateBackend(this, config);
+	}
+
+	// ------------------------------------------------------------------------
+	//  checkpoint state persistence
+	// ------------------------------------------------------------------------
+
 	@Override
 	public OperatorStateBackend createOperatorStateBackend(
 		Environment env,
@@ -98,12 +285,7 @@ public class MemoryStateBackend extends AbstractStateBackend {
 		return new DefaultOperatorStateBackend(
 			env.getUserClassLoader(),
 			env.getExecutionConfig(),
-			asynchronousSnapshots);
-	}
-
-	@Override
-	public String toString() {
-		return "MemoryStateBackend (data in heap memory / checkpoints to JobManager)";
+			isUsingAsynchronousSnapshots());
 	}
 
 	@Override
@@ -121,6 +303,10 @@ public class MemoryStateBackend extends AbstractStateBackend {
 		return new MemCheckpointStreamFactory(maxStateSize);
 	}
 
+	// ------------------------------------------------------------------------
+	//  checkpoint state persistence
+	// ------------------------------------------------------------------------
+
 	@Override
 	public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
 			Environment env, JobID jobID,
@@ -136,7 +322,20 @@ public class MemoryStateBackend extends AbstractStateBackend {
 				env.getUserClassLoader(),
 				numberOfKeyGroups,
 				keyGroupRange,
-				asynchronousSnapshots,
+				isUsingAsynchronousSnapshots(),
 				env.getExecutionConfig());
 	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "MemoryStateBackend (data in heap memory / checkpoints to JobManager) " +
+				"(checkpoints: '" + getCheckpointPath() +
+				"', savepoints: '" + getSavepointPath() +
+				"', asynchronous: " + asynchronousSnapshots +
+				", maxStateSize: " + maxStateSize + ")";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fa03e78d/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java
new file mode 100644
index 0000000..10b6c20
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.memory;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.StateBackendFactory;
+
+/**
+ * A factory that creates an {@link MemoryStateBackend} from a configuration.
+ */
+@PublicEvolving
+public class MemoryStateBackendFactory implements StateBackendFactory<MemoryStateBackend> {
+
+	@Override
+	public MemoryStateBackend createFromConfig(Configuration config) {
+		return new MemoryStateBackend().configure(config);
+	}
+}
\ No newline at end of file


[15/17] flink git commit: [hotfix] [core] Move 'ThrowingConsumer' and 'RunnableWithException' to proper package (.util.function)

Posted by se...@apache.org.
[hotfix] [core] Move 'ThrowingConsumer' and 'RunnableWithException' to proper package (.util.function)

This also adds missing stability annotations to the functional interfaces in 'util.function'.


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

Branch: refs/heads/master
Commit: 1e6a91a3bf7e734eccdb034ce505b3775b709265
Parents: edc6f10
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jan 18 14:15:49 2018 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:04 2018 +0100

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |  7 +---
 .../java/org/apache/flink/util/LambdaUtil.java  |  2 +
 .../org/apache/flink/util/ThrowingConsumer.java | 37 ------------------
 .../util/function/RunnableWithException.java    | 38 +++++++++++++++++++
 .../util/function/SupplierWithException.java    |  3 ++
 .../flink/util/function/ThrowingConsumer.java   | 40 ++++++++++++++++++++
 6 files changed, 84 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/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 e36ea2b..6e0a9c6 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
@@ -25,6 +25,7 @@
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.RunnableWithException;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
@@ -396,12 +397,6 @@ public final class ExceptionUtils {
 		}
 	}
 
-	@FunctionalInterface
-	public interface RunnableWithException {
-
-		void run() throws Exception;
-	}
-
 	// ------------------------------------------------------------------------
 
 	/** Private constructor to prevent instantiation. */

http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
index 8ac0f0e..bce403a 100644
--- a/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.util.function.ThrowingConsumer;
+
 /**
  * This class offers utility functions for Java's lambda features.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java b/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
deleted file mode 100644
index a180a12..0000000
--- a/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.util;
-
-/**
- * This interface is basically Java's {@link java.util.function.Consumer} interface enhanced with the ability to throw
- * an exception.
- *
- * @param <T> type of the consumed elements.
- */
-@FunctionalInterface
-public interface ThrowingConsumer<T> {
-
-	/**
-	 * Performs this operation on the given argument.
-	 *
-	 * @param t the input argument
-	 * @throws Exception on errors during consumption
-	 */
-	void accept(T t) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/flink-core/src/main/java/org/apache/flink/util/function/RunnableWithException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/function/RunnableWithException.java b/flink-core/src/main/java/org/apache/flink/util/function/RunnableWithException.java
new file mode 100644
index 0000000..6270726
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/function/RunnableWithException.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.util.function;
+
+import org.apache.flink.annotation.Public;
+
+/**
+ * Similar to a {@link Runnable}, this interface is used to capture a block of code
+ * to be executed. In contrast to {@code Runnable}, this interface allows throwing
+ * checked exceptions.
+ */
+@Public
+@FunctionalInterface
+public interface RunnableWithException {
+
+	/**
+	 * The work method.
+	 *
+	 * @throws Exception Exceptions may be thrown.
+	 */
+	void run() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java b/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java
index 63be9bf..f584887 100644
--- a/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.util.function;
 
+import org.apache.flink.annotation.Public;
+
 /**
  * A functional interface for a {@link java.util.function.Supplier} that may
  * throw exceptions.
@@ -25,6 +27,7 @@ package org.apache.flink.util.function;
  * @param <R> The type of the result of the supplier.
  * @param <E> The type of Exceptions thrown by this function.
  */
+@Public
 @FunctionalInterface
 public interface SupplierWithException<R, E extends Throwable> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1e6a91a3/flink-core/src/main/java/org/apache/flink/util/function/ThrowingConsumer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/function/ThrowingConsumer.java b/flink-core/src/main/java/org/apache/flink/util/function/ThrowingConsumer.java
new file mode 100644
index 0000000..1452071
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/function/ThrowingConsumer.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.util.function;
+
+import org.apache.flink.annotation.Public;
+
+/**
+ * This interface is basically Java's {@link java.util.function.Consumer} interface enhanced with the ability to throw
+ * an exception.
+ *
+ * @param <T> type of the consumed elements.
+ */
+@Public
+@FunctionalInterface
+public interface ThrowingConsumer<T> {
+
+	/**
+	 * Performs this operation on the given argument.
+	 *
+	 * @param t the input argument
+	 * @throws Exception on errors during consumption
+	 */
+	void accept(T t) throws Exception;
+}


[17/17] flink git commit: [hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.util'

Posted by se...@apache.org.
[hotfix] [core] Fix checkstyle for 'flink-core' : 'org.apache.flink.util'


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

Branch: refs/heads/master
Commit: 212ee3d430190e6e771c79a9b94fd8410675a534
Parents: 1e6a91a
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jan 18 14:41:05 2018 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:04 2018 +0100

----------------------------------------------------------------------
 .../flink/util/AbstractCloseableRegistry.java   |  4 +-
 .../java/org/apache/flink/util/AbstractID.java  | 22 ++---
 .../org/apache/flink/util/CollectionUtil.java   |  3 +
 .../java/org/apache/flink/util/Collector.java   |  7 +-
 .../flink/util/DynamicCodeLoadingException.java |  4 +-
 .../org/apache/flink/util/ExceptionUtils.java   | 31 +++----
 .../java/org/apache/flink/util/FileUtils.java   | 36 ++++----
 .../org/apache/flink/util/FlinkException.java   |  8 +-
 .../flink/util/FlinkRuntimeException.java       |  8 +-
 .../java/org/apache/flink/util/FutureUtil.java  |  3 +
 .../java/org/apache/flink/util/IOUtils.java     | 27 +++---
 .../apache/flink/util/InstantiationUtil.java    | 93 ++++++++++----------
 .../org/apache/flink/util/IterableIterator.java |  3 +-
 .../flink/util/LongValueSequenceIterator.java   | 17 ++--
 .../java/org/apache/flink/util/MathUtils.java   | 28 +++---
 .../flink/util/MutableObjectIterator.java       | 17 ++--
 .../java/org/apache/flink/util/NetUtils.java    | 57 ++++++------
 .../flink/util/NumberSequenceIterator.java      | 59 ++++++-------
 .../org/apache/flink/util/OperatingSystem.java  | 34 +++----
 .../java/org/apache/flink/util/OutputTag.java   |  8 +-
 .../org/apache/flink/util/Preconditions.java    | 42 ++++-----
 .../org/apache/flink/util/PropertiesUtil.java   |  9 +-
 .../org/apache/flink/util/ReflectionUtil.java   | 25 +++---
 .../apache/flink/util/SerializableObject.java   |  2 +-
 .../apache/flink/util/SerializedThrowable.java  | 32 +++----
 .../org/apache/flink/util/SerializedValue.java  |  6 +-
 .../apache/flink/util/SplittableIterator.java   | 11 +--
 .../flink/util/StateMigrationException.java     |  2 +-
 .../org/apache/flink/util/StringBasedID.java    |  2 +-
 .../java/org/apache/flink/util/StringUtils.java | 72 ++++++++-------
 .../org/apache/flink/util/StringValueUtils.java | 50 +++++------
 .../org/apache/flink/util/UnionIterator.java    | 27 ++++--
 .../java/org/apache/flink/util/Visitable.java   | 17 ++--
 .../java/org/apache/flink/util/Visitor.java     | 12 ++-
 .../org/apache/flink/util/WrappingProxy.java    |  7 +-
 .../apache/flink/util/WrappingProxyUtil.java    |  3 +
 .../flink/util/WrappingRuntimeException.java    |  4 +-
 .../apache/flink/util/ExceptionUtilsTest.java   |  6 +-
 .../org/apache/flink/util/FileUtilsTest.java    |  2 +-
 .../flink/util/InstantiationUtilTest.java       | 39 ++++----
 .../util/LongValueSequenceIteratorTest.java     | 19 ++--
 .../org/apache/flink/util/MathUtilTest.java     | 17 ++--
 .../org/apache/flink/util/NetUtilsTest.java     | 58 +++++++-----
 .../flink/util/NumberSequenceIteratorTest.java  | 41 ++++-----
 .../org/apache/flink/util/OutputTagTest.java    |  5 ++
 .../apache/flink/util/ResourceGuardTest.java    | 19 ++--
 .../apache/flink/util/SerializedValueTest.java  |  8 +-
 .../org/apache/flink/util/StringUtilsTest.java  | 14 +--
 .../apache/flink/util/StringValueUtilsTest.java | 16 ++--
 .../apache/flink/util/UnionIteratorTest.java    | 26 +++---
 tools/maven/suppressions-core.xml               |  3 -
 51 files changed, 568 insertions(+), 497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
index 14e765c..e6589f6 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractCloseableRegistry.java
@@ -33,9 +33,9 @@ import java.util.Map;
 /**
  * This is the abstract base class for registries that allow to register instances of {@link Closeable}, which are all
  * closed if this registry is closed.
- * 
+ *
  * <p>Registering to an already closed registry will throw an exception and close the provided {@link Closeable}
- * 
+ *
  * <p>All methods in this class are thread-safe.
  *
  * @param <C> Type of the closeable this registers

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
index 12d634d..41083c6 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.util;
 
-import java.util.Random;
-
 import org.apache.flink.annotation.PublicEvolving;
 
+import java.util.Random;
+
 /**
  * A statistically unique identification number.
  */
@@ -32,25 +32,25 @@ public class AbstractID implements Comparable<AbstractID>, java.io.Serializable
 
 	private static final Random RND = new Random();
 
-	/** The size of a long in bytes */
+	/** The size of a long in bytes. */
 	private static final int SIZE_OF_LONG = 8;
 
-	/** The size of the ID in byte */
+	/** The size of the ID in byte. */
 	public static final int SIZE = 2 * SIZE_OF_LONG;
 
 	// ------------------------------------------------------------------------
 
-	/** The upper part of the actual ID */
+	/** The upper part of the actual ID. */
 	protected final long upperPart;
 
-	/** The lower part of the actual ID */
+	/** The lower part of the actual ID. */
 	protected final long lowerPart;
 
-	/** The memoized value returned by toString() */
+	/** The memoized value returned by toString(). */
 	private String toString;
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Constructs a new ID with a specific bytes value.
 	 */
@@ -130,7 +130,7 @@ public class AbstractID implements Comparable<AbstractID>, java.io.Serializable
 	// --------------------------------------------------------------------------------------------
 	//  Standard Utilities
 	// --------------------------------------------------------------------------------------------
-	
+
 	@Override
 	public boolean equals(Object obj) {
 		if (obj == this) {
@@ -166,8 +166,8 @@ public class AbstractID implements Comparable<AbstractID>, java.io.Serializable
 
 	@Override
 	public int compareTo(AbstractID o) {
-		int diff1 = (this.upperPart < o.upperPart) ? -1 : ((this.upperPart == o.upperPart) ? 0 : 1);
-		int diff2 = (this.lowerPart < o.lowerPart) ? -1 : ((this.lowerPart == o.lowerPart) ? 0 : 1);
+		int diff1 = Long.compare(this.upperPart, o.upperPart);
+		int diff2 = Long.compare(this.lowerPart, o.lowerPart);
 		return diff1 == 0 ? diff2 : diff1;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
index bd6847a..072d905 100644
--- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
@@ -23,6 +23,9 @@ import org.apache.flink.annotation.Internal;
 import java.util.Collection;
 import java.util.Map;
 
+/**
+ * Simple utility to work with Java collections.
+ */
 @Internal
 public final class CollectionUtil {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/Collector.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Collector.java b/flink-core/src/main/java/org/apache/flink/util/Collector.java
index 09f5c69..003459c 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Collector.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Collector.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Public;
@@ -27,14 +26,14 @@ import org.apache.flink.annotation.Public;
  */
 @Public
 public interface Collector<T> {
-	
+
 	/**
 	 * Emits a record.
-	 * 
+	 *
 	 * @param record The record to collect.
 	 */
 	void collect(T record);
-	
+
 	/**
 	 * Closes the collector. If any data was buffered, that data will be flushed.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java b/flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java
index d18b9d3..c9d2234 100644
--- a/flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java
@@ -22,10 +22,10 @@ import org.apache.flink.annotation.Public;
 
 /**
  * An exception that is thrown if the dynamic instantiation of code fails.
- * 
+ *
  * <p>This exception is supposed to "sum up" the zoo of exceptions typically thrown around
  * dynamic code loading and instantiations:
- * 
+ *
  * <pre>{@code
  * try {
  *     Class.forName(classname).asSubclass(TheType.class).newInstance();

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/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 6e0a9c6..5c69564 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
@@ -19,7 +19,7 @@
 //
 // The function "stringifyException" is based on source code from the Hadoop Project (http://hadoop.apache.org/),
 // licensed by the Apache Software Foundation (ASF) under the Apache License, Version 2.0.
-// See the NOTICE file distributed with this work for additional information regarding copyright ownership. 
+// See the NOTICE file distributed with this work for additional information regarding copyright ownership.
 //
 
 package org.apache.flink.util;
@@ -28,6 +28,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.function.RunnableWithException;
 
 import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -43,15 +44,15 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 @Internal
 public final class ExceptionUtils {
 
-	/** The stringified representation of a null exception reference */ 
+	/** The stringified representation of a null exception reference. */
 	public static final String STRINGIFIED_NULL_EXCEPTION = "(null)";
 
 	/**
 	 * Makes a string representation of the exception's stack trace, or "(null)", if the
 	 * exception is null.
-	 * 
-	 * This method makes a best effort and never fails.
-	 * 
+	 *
+	 * <p>This method makes a best effort and never fails.
+	 *
 	 * @param e The exception to stringify.
 	 * @return A string with exception name and call stack.
 	 */
@@ -59,7 +60,7 @@ public final class ExceptionUtils {
 		if (e == null) {
 			return STRINGIFIED_NULL_EXCEPTION;
 		}
-		
+
 		try {
 			StringWriter stm = new StringWriter();
 			PrintWriter wrt = new PrintWriter(stm);
@@ -91,12 +92,12 @@ public final class ExceptionUtils {
 	/**
 	 * Checks whether the given exception indicates a situation that may leave the
 	 * JVM in a corrupted state, or an out-of-memory error.
-	 * 
+	 *
 	 * <p>See {@link ExceptionUtils#isJvmFatalError(Throwable)} for a list of fatal JVM errors.
 	 * This method additionally classifies the {@link OutOfMemoryError} as fatal, because it
 	 * may occur in any thread (not the one that allocated the majority of the memory) and thus
 	 * is often not recoverable by destroying the particular thread that threw the exception.
-	 * 
+	 *
 	 * @param t The exception to check.
 	 * @return True, if the exception is fatal to the JVM or and OutOfMemoryError, false otherwise.
 	 */
@@ -107,7 +108,7 @@ public final class ExceptionUtils {
 	/**
 	 * Rethrows the given {@code Throwable}, if it represents an error that is fatal to the JVM.
 	 * See {@link ExceptionUtils#isJvmFatalError(Throwable)} for a definition of fatal errors.
-	 * 
+	 *
 	 * @param t The Throwable to check and rethrow.
 	 */
 	public static void rethrowIfFatalError(Throwable t) {
@@ -180,7 +181,7 @@ public final class ExceptionUtils {
 	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
 	 * throw an arbitrary Throwable. Errors and RuntimeExceptions are thrown directly, other exceptions
 	 * are packed into runtime exceptions
-	 * 
+	 *
 	 * @param t The throwable to be thrown.
 	 */
 	public static void rethrow(Throwable t) {
@@ -194,12 +195,12 @@ public final class ExceptionUtils {
 			throw new RuntimeException(t);
 		}
 	}
-	
+
 	/**
 	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
 	 * throw an arbitrary Throwable. Errors and RuntimeExceptions are thrown directly, other exceptions
 	 * are packed into a parent RuntimeException.
-	 * 
+	 *
 	 * @param t The throwable to be thrown.
 	 * @param parentMessage The message for the parent RuntimeException, if one is needed.
 	 */
@@ -257,10 +258,10 @@ public final class ExceptionUtils {
 	/**
 	 * Re-throws the given {@code Throwable} in scenarios where the signatures allows only IOExceptions
 	 * (and RuntimeException and Error).
-	 * 
-	 * Throws this exception directly, if it is an IOException, a RuntimeException, or an Error. Otherwise it 
+	 *
+	 * <p>Throws this exception directly, if it is an IOException, a RuntimeException, or an Error. Otherwise it
 	 * wraps it in an IOException and throws it.
-	 * 
+	 *
 	 * @param t The Throwable to be thrown.
 	 */
 	public static void rethrowIOException(Throwable t) throws IOException {

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
index a7afb43..6d15b2e 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
@@ -39,7 +39,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 public final class FileUtils {
 
 	/** The alphabet to construct the random part of the filename from. */
-	private static final char[] ALPHABET = 
+	private static final char[] ALPHABET =
 			{ '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 'b', 'c', 'd', 'e', 'f' };
 
 	/** The length of the random part of the filename. */
@@ -50,7 +50,7 @@ public final class FileUtils {
 	/**
 	 * Constructs a random filename with the given prefix and
 	 * a random part generated from hex characters.
-	 * 
+	 *
 	 * @param prefix
 	 *        the prefix to the filename to be constructed
 	 * @return the generated random filename with the given prefix
@@ -65,11 +65,11 @@ public final class FileUtils {
 
 		return stringBuilder.toString();
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Simple reading and writing of files
 	// ------------------------------------------------------------------------
-	
+
 	public static String readFile(File file, String charsetName) throws IOException {
 		byte[] bytes = Files.readAllBytes(file.toPath());
 		return new String(bytes, charsetName);
@@ -83,7 +83,7 @@ public final class FileUtils {
 		byte[] bytes = contents.getBytes(encoding);
 		Files.write(file.toPath(), bytes, StandardOpenOption.WRITE);
 	}
-	
+
 	public static void writeFileUtf8(File file, String contents) throws IOException {
 		writeFile(file, contents, "UTF-8");
 	}
@@ -94,14 +94,14 @@ public final class FileUtils {
 
 	/**
 	 * Removes the given file or directory recursively.
-	 * 
+	 *
 	 * <p>If the file or directory does not exist, this does not throw an exception, but simply does nothing.
 	 * It considers the fact that a file-to-be-deleted is not present a success.
-	 * 
+	 *
 	 * <p>This method is safe against other concurrent deletion attempts.
-	 * 
+	 *
 	 * @param file The file or directory to delete.
-	 * 
+	 *
 	 * @throws IOException Thrown if the directory could not be cleaned for some reason, for example
 	 *                     due to missing access/write permissions.
 	 */
@@ -125,12 +125,12 @@ public final class FileUtils {
 
 	/**
 	 * Deletes the given directory recursively.
-	 * 
+	 *
 	 * <p>If the directory does not exist, this does not throw an exception, but simply does nothing.
 	 * It considers the fact that a directory-to-be-deleted is not present a success.
-	 * 
+	 *
 	 * <p>This method is safe against other concurrent deletion attempts.
-	 * 
+	 *
 	 * @param directory The directory to be deleted.
 	 * @throws IOException Thrown if the given file is not a directory, or if the directory could not be
 	 *                     deleted for some reason, for example due to missing access/write permissions.
@@ -171,10 +171,10 @@ public final class FileUtils {
 	/**
 	 * Deletes the given directory recursively, not reporting any I/O exceptions
 	 * that occur.
-	 * 
+	 *
 	 * <p>This method is identical to {@link FileUtils#deleteDirectory(File)}, except that it
 	 * swallows all exceptions and may leave the job quietly incomplete.
-	 * 
+	 *
 	 * @param directory The directory to delete.
 	 */
 	public static void deleteDirectoryQuietly(File directory) {
@@ -189,12 +189,12 @@ public final class FileUtils {
 	}
 
 	/**
-	 * Removes all files contained within a directory, without removing the directory itself. 
-	 * 
+	 * Removes all files contained within a directory, without removing the directory itself.
+	 *
 	 * <p>This method is safe against other concurrent deletion attempts.
-	 * 
+	 *
 	 * @param directory The directory to remove all files from.
-	 * 
+	 *
 	 * @throws FileNotFoundException Thrown if the directory itself does not exist.
 	 * @throws IOException Thrown if the file indicates a proper file and not a directory, or if
 	 *                     the directory could not be cleaned for some reason, for example

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/FlinkException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/FlinkException.java b/flink-core/src/main/java/org/apache/flink/util/FlinkException.java
index 550ab2c..9f34c93 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FlinkException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FlinkException.java
@@ -30,7 +30,7 @@ public class FlinkException extends Exception {
 
 	/**
 	 * Creates a new Exception with the given message and null as the cause.
-	 * 
+	 *
 	 * @param message The exception message
 	 */
 	public FlinkException(String message) {
@@ -39,7 +39,7 @@ public class FlinkException extends Exception {
 
 	/**
 	 * Creates a new exception with a null message and the given cause.
-	 * 
+	 *
 	 * @param cause The exception that caused this exception
 	 */
 	public FlinkException(Throwable cause) {
@@ -47,8 +47,8 @@ public class FlinkException extends Exception {
 	}
 
 	/**
-	 * Creates a new exception with the given message and cause
-	 * 
+	 * Creates a new exception with the given message and cause.
+	 *
 	 * @param message The exception message
 	 * @param cause The exception that caused this exception
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java b/flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java
index 16b783b..cfee197 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java
@@ -30,7 +30,7 @@ public class FlinkRuntimeException extends RuntimeException {
 
 	/**
 	 * Creates a new Exception with the given message and null as the cause.
-	 * 
+	 *
 	 * @param message The exception message
 	 */
 	public FlinkRuntimeException(String message) {
@@ -39,7 +39,7 @@ public class FlinkRuntimeException extends RuntimeException {
 
 	/**
 	 * Creates a new exception with a null message and the given cause.
-	 * 
+	 *
 	 * @param cause The exception that caused this exception
 	 */
 	public FlinkRuntimeException(Throwable cause) {
@@ -47,8 +47,8 @@ public class FlinkRuntimeException extends RuntimeException {
 	}
 
 	/**
-	 * Creates a new exception with the given message and cause
-	 * 
+	 * Creates a new exception with the given message and cause.
+	 *
 	 * @param message The exception message
 	 * @param cause The exception that caused this exception
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java b/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
index 5ecf344..b6bac88 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FutureUtil.java
@@ -23,6 +23,9 @@ import org.apache.flink.annotation.Internal;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.RunnableFuture;
 
+/**
+ * Simple utility class to work with Java's Futures.
+ */
 @Internal
 public class FutureUtil {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/IOUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/IOUtils.java b/flink-core/src/main/java/org/apache/flink/util/IOUtils.java
index 0bdc13a..498c899 100644
--- a/flink-core/src/main/java/org/apache/flink/util/IOUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/IOUtils.java
@@ -28,20 +28,19 @@ import java.net.Socket;
 
 /**
  * An utility class for I/O related functionality.
- * 
  */
 public final class IOUtils {
 
 	/** The block size for byte operations in byte. */
 	private static final int BLOCKSIZE = 4096;
-	
+
 	// ------------------------------------------------------------------------
 	//  Byte copy operations
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Copies from one stream to another.
-	 * 
+	 *
 	 * @param in
 	 *        InputStream to read from
 	 * @param out
@@ -80,7 +79,7 @@ public final class IOUtils {
 	/**
 	 * Copies from one stream to another. <strong>closes the input and output
 	 * streams at the end</strong>.
-	 * 
+	 *
 	 * @param in
 	 *        InputStream to read from
 	 * @param out
@@ -94,7 +93,7 @@ public final class IOUtils {
 
 	/**
 	 * Copies from one stream to another.
-	 * 
+	 *
 	 * @param in
 	 *        InputStream to read from
 	 * @param out
@@ -112,10 +111,10 @@ public final class IOUtils {
 	// ------------------------------------------------------------------------
 	//  Stream input skipping
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Reads len bytes in a loop.
-	 * 
+	 *
 	 * @param in
 	 *        The InputStream to read from
 	 * @param buf
@@ -142,7 +141,7 @@ public final class IOUtils {
 
 	/**
 	 * Similar to readFully(). Skips bytes in a loop.
-	 * 
+	 *
 	 * @param in
 	 *        The InputStream to skip bytes from
 	 * @param len
@@ -163,11 +162,11 @@ public final class IOUtils {
 	// ------------------------------------------------------------------------
 	//  Silent I/O cleanup / closing
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Close the AutoCloseable objects and <b>ignore</b> any {@link Exception} or
 	 * null pointers. Must only be used for cleanup in exception handlers.
-	 * 
+	 *
 	 * @param log
 	 *        the log to record problems to at debug level. Can be <code>null</code>.
 	 * @param closeables
@@ -190,7 +189,7 @@ public final class IOUtils {
 	/**
 	 * Closes the stream ignoring {@link IOException}. Must only be called in
 	 * cleaning up from exception handlers.
-	 * 
+	 *
 	 * @param stream
 	 *        the stream to close
 	 */
@@ -200,7 +199,7 @@ public final class IOUtils {
 
 	/**
 	 * Closes the socket ignoring {@link IOException}.
-	 * 
+	 *
 	 * @param sock
 	 *        the socket to close
 	 */
@@ -254,6 +253,8 @@ public final class IOUtils {
 	}
 
 	/**
+	 * Closes the given AutoCloseable.
+	 *
 	 * <p><b>Important:</b> This method is expected to never throw an exception.
 	 */
 	public static void closeQuietly(AutoCloseable closeable) {
@@ -263,7 +264,7 @@ public final class IOUtils {
 			}
 		} catch (Throwable ignored) {}
 	}
-	
+
 	// ------------------------------------------------------------------------
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
index 6441c86..4bfec2e 100644
--- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
@@ -43,7 +43,7 @@ import java.util.HashMap;
  */
 @Internal
 public final class InstantiationUtil {
-	
+
 	/**
 	 * A custom ObjectInputStream that can load classes using a specific ClassLoader.
 	 */
@@ -77,11 +77,11 @@ public final class InstantiationUtil {
 
 			return super.resolveClass(desc);
 		}
-		
+
 		// ------------------------------------------------
 
 		private static final HashMap<String, Class<?>> primitiveClasses = new HashMap<>(9);
-		
+
 		static {
 			primitiveClasses.put("boolean", boolean.class);
 			primitiveClasses.put("byte", byte.class);
@@ -94,17 +94,17 @@ public final class InstantiationUtil {
 			primitiveClasses.put("void", void.class);
 		}
 	}
-	
-	
+
+
 	/**
 	 * Creates a new instance of the given class.
-	 * 
+	 *
 	 * @param <T> The generic type of the class.
 	 * @param clazz The class to instantiate.
 	 * @param castTo Optional parameter, specifying the class that the given class must be a subclass off. This
-	 *               argument is added to prevent class cast exceptions occurring later. 
+	 *               argument is added to prevent class cast exceptions occurring later.
 	 * @return An instance of the given class.
-	 * 
+	 *
 	 * @throws RuntimeException Thrown, if the class could not be instantiated. The exception contains a detailed
 	 *                          message about the reason why the instantiation failed.
 	 */
@@ -112,24 +112,24 @@ public final class InstantiationUtil {
 		if (clazz == null) {
 			throw new NullPointerException();
 		}
-		
+
 		// check if the class is a subclass, if the check is required
 		if (castTo != null && !castTo.isAssignableFrom(clazz)) {
-			throw new RuntimeException("The class '" + clazz.getName() + "' is not a subclass of '" + 
+			throw new RuntimeException("The class '" + clazz.getName() + "' is not a subclass of '" +
 				castTo.getName() + "' as is required.");
 		}
-		
+
 		return instantiate(clazz);
 	}
 
 	/**
 	 * Creates a new instance of the given class.
-	 * 
+	 *
 	 * @param <T> The generic type of the class.
 	 * @param clazz The class to instantiate.
 
 	 * @return An instance of the given class.
-	 * 
+	 *
 	 * @throws RuntimeException Thrown, if the class could not be instantiated. The exception contains a detailed
 	 *                          message about the reason why the instantiation failed.
 	 */
@@ -137,7 +137,7 @@ public final class InstantiationUtil {
 		if (clazz == null) {
 			throw new NullPointerException();
 		}
-		
+
 		// try to instantiate the class
 		try {
 			return clazz.newInstance();
@@ -145,23 +145,23 @@ public final class InstantiationUtil {
 		catch (InstantiationException | IllegalAccessException iex) {
 			// check for the common problem causes
 			checkForInstantiation(clazz);
-			
+
 			// here we are, if non of the common causes was the problem. then the error was
 			// most likely an exception in the constructor or field initialization
-			throw new RuntimeException("Could not instantiate type '" + clazz.getName() + 
+			throw new RuntimeException("Could not instantiate type '" + clazz.getName() +
 					"' due to an unspecified exception: " + iex.getMessage(), iex);
 		}
 		catch (Throwable t) {
 			String message = t.getMessage();
-			throw new RuntimeException("Could not instantiate type '" + clazz.getName() + 
-				"' Most likely the constructor (or a member variable initialization) threw an exception" + 
+			throw new RuntimeException("Could not instantiate type '" + clazz.getName() +
+				"' Most likely the constructor (or a member variable initialization) threw an exception" +
 				(message == null ? "." : ": " + message), t);
 		}
 	}
-	
+
 	/**
 	 * Checks, whether the given class has a public nullary constructor.
-	 * 
+	 *
 	 * @param clazz The class to check.
 	 * @return True, if the class has a public nullary constructor, false if not.
 	 */
@@ -175,20 +175,20 @@ public final class InstantiationUtil {
 		}
 		return false;
 	}
-	
+
 	/**
 	 * Checks, whether the given class is public.
-	 * 
+	 *
 	 * @param clazz The class to check.
 	 * @return True, if the class is public, false if not.
 	 */
 	public static boolean isPublic(Class<?> clazz) {
 		return Modifier.isPublic(clazz.getModifiers());
 	}
-	
+
 	/**
 	 * Checks, whether the class is a proper class, i.e. not abstract or an interface, and not a primitive type.
-	 * 
+	 *
 	 * @param clazz The class to check.
 	 * @return True, if the class is a proper class, false otherwise.
 	 */
@@ -200,29 +200,29 @@ public final class InstantiationUtil {
 	/**
 	 * Checks, whether the class is an inner class that is not statically accessible. That is especially true for
 	 * anonymous inner classes.
-	 * 
+	 *
 	 * @param clazz The class to check.
 	 * @return True, if the class is a non-statically accessible inner class.
 	 */
 	public static boolean isNonStaticInnerClass(Class<?> clazz) {
-		return clazz.getEnclosingClass() != null && 
+		return clazz.getEnclosingClass() != null &&
 			(clazz.getDeclaringClass() == null || !Modifier.isStatic(clazz.getModifiers()));
 	}
-	
+
 	/**
 	 * Performs a standard check whether the class can be instantiated by {@code Class#newInstance()}.
-	 * 
+	 *
 	 * @param clazz The class to check.
 	 * @throws RuntimeException Thrown, if the class cannot be instantiated by {@code Class#newInstance()}.
 	 */
 	public static void checkForInstantiation(Class<?> clazz) {
 		final String errorMessage = checkForInstantiationError(clazz);
-		
+
 		if (errorMessage != null) {
 			throw new RuntimeException("The class '" + clazz.getName() + "' is not instantiable: " + errorMessage);
 		}
 	}
-	
+
 	public static String checkForInstantiationError(Class<?> clazz) {
 		if (!isPublic(clazz)) {
 			return "The class is not public.";
@@ -235,19 +235,19 @@ public final class InstantiationUtil {
 		} else if (!hasPublicNullaryConstructor(clazz)) {
 			return "The class has no (implicit) public nullary constructor, i.e. a constructor without arguments.";
 		} else {
-			return null; 
+			return null;
 		}
 	}
-	
+
 	public static <T> T readObjectFromConfig(Configuration config, String key, ClassLoader cl) throws IOException, ClassNotFoundException {
 		byte[] bytes = config.getBytes(key, null);
 		if (bytes == null) {
 			return null;
 		}
-		
+
 		return deserializeObject(bytes, cl);
 	}
-	
+
 	public static void writeObjectToConfig(Object o, Configuration config, String key) throws IOException {
 		byte[] bytes = serializeObject(o);
 		config.setBytes(key, bytes);
@@ -281,7 +281,7 @@ public final class InstantiationUtil {
 		DataInputViewStreamWrapper inputViewWrapper = new DataInputViewStreamWrapper(new ByteArrayInputStream(buf));
 		return serializer.deserialize(reuse, inputViewWrapper);
 	}
-	
+
 	@SuppressWarnings("unchecked")
 	public static <T> T deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException {
 		final ClassLoader old = Thread.currentThread().getContextClassLoader();
@@ -338,9 +338,11 @@ public final class InstantiationUtil {
 	 *
 	 * @param obj Object to clone
 	 * @param <T> Type of the object to clone
-	 * @return Cloned object
-	 * @throws IOException
-	 * @throws ClassNotFoundException
+	 * @return The cloned object
+	 *
+	 * @throws IOException Thrown if the serialization or deserialization process fails.
+	 * @throws ClassNotFoundException Thrown if any of the classes referenced by the object
+	 *                                cannot be resolved during deserialization.
 	 */
 	public static <T extends Serializable> T clone(T obj) throws IOException, ClassNotFoundException {
 		if (obj == null) {
@@ -357,11 +359,12 @@ public final class InstantiationUtil {
 	 * @param obj Object to clone
 	 * @param classLoader The classloader to resolve the classes during deserialization.
 	 * @param <T> Type of the object to clone
-	 * 
+	 *
 	 * @return Cloned object
-	 * 
-	 * @throws IOException
-	 * @throws ClassNotFoundException
+	 *
+	 * @throws IOException Thrown if the serialization or deserialization process fails.
+	 * @throws ClassNotFoundException Thrown if any of the classes referenced by the object
+	 *                                cannot be resolved during deserialization.
 	 */
 	public static <T extends Serializable> T clone(T obj, ClassLoader classLoader) throws IOException, ClassNotFoundException {
 		if (obj == null) {
@@ -399,10 +402,10 @@ public final class InstantiationUtil {
 			return copy;
 		}
 	}
-	
-	
+
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Private constructor to prevent instantiation.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/IterableIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/IterableIterator.java b/flink-core/src/main/java/org/apache/flink/util/IterableIterator.java
index f66bab9..322459f 100644
--- a/flink-core/src/main/java/org/apache/flink/util/IterableIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/IterableIterator.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
@@ -25,7 +24,7 @@ import java.util.Iterator;
 
 /**
  * An {@link Iterator} that is also {@link Iterable} (often by returning itself).
- * 
+ *
  * @param <E> The iterated elements' type.
  */
 @Internal

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/LongValueSequenceIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/LongValueSequenceIterator.java b/flink-core/src/main/java/org/apache/flink/util/LongValueSequenceIterator.java
index 86a8ce6..7c9fea7 100644
--- a/flink-core/src/main/java/org/apache/flink/util/LongValueSequenceIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/LongValueSequenceIterator.java
@@ -33,13 +33,13 @@ public class LongValueSequenceIterator extends SplittableIterator<LongValue> {
 
 	private static final long serialVersionUID = 1L;
 
-	/** The last number returned by the iterator */
+	/** The last number returned by the iterator. */
 	private final long to;
 
-	/** The next number to be returned */
+	/** The next number to be returned. */
 	private long current;
 
-	/** The next value to be returned */
+	/** The next value to be returned. */
 	private LongValue currentValue = new LongValue();
 
 	/**
@@ -125,7 +125,7 @@ public class LongValueSequenceIterator extends SplittableIterator<LongValue> {
 
 			if (current == Long.MIN_VALUE) {
 				// this means to >= 0
-				halfDiff = (Long.MAX_VALUE/2+1) + to/2;
+				halfDiff = (Long.MAX_VALUE / 2 + 1) + to / 2;
 			} else {
 				long posFrom = -current;
 				if (posFrom > to) {
@@ -156,12 +156,12 @@ public class LongValueSequenceIterator extends SplittableIterator<LongValue> {
 			int i = 0;
 			for (; i < numWithExtra; i++) {
 				long next = curr + elementsPerSplit + 1;
-				iters[i] = new LongValueSequenceIterator(curr, next-1);
+				iters[i] = new LongValueSequenceIterator(curr, next - 1);
 				curr = next;
 			}
 			for (; i < numPartitions; i++) {
 				long next = curr + elementsPerSplit;
-				iters[i] = new LongValueSequenceIterator(curr, next-1, true);
+				iters[i] = new LongValueSequenceIterator(curr, next - 1, true);
 				curr = next;
 			}
 
@@ -180,14 +180,13 @@ public class LongValueSequenceIterator extends SplittableIterator<LongValue> {
 		}
 	}
 
-
 	@Override
 	public int getMaximumNumberOfSplits() {
-		if (to >= Integer.MAX_VALUE || current <= Integer.MIN_VALUE || to-current+1 >= Integer.MAX_VALUE) {
+		if (to >= Integer.MAX_VALUE || current <= Integer.MIN_VALUE || to - current + 1 >= Integer.MAX_VALUE) {
 			return Integer.MAX_VALUE;
 		}
 		else {
-			return (int) (to-current+1);
+			return (int) (to - current + 1);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/MathUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/MathUtils.java b/flink-core/src/main/java/org/apache/flink/util/MathUtils.java
index 1d84a39..48c32a3 100644
--- a/flink-core/src/main/java/org/apache/flink/util/MathUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/MathUtils.java
@@ -22,13 +22,13 @@ package org.apache.flink.util;
  * Collection of simple mathematical routines.
  */
 public final class MathUtils {
-	
+
 	/**
 	 * Computes the logarithm of the given value to the base of 2, rounded down. It corresponds to the
 	 * position of the highest non-zero bit. The position is counted, starting with 0 from the least
 	 * significant bit to the most significant bit. For example, <code>log2floor(16) = 4</code>, and
 	 * <code>log2floor(10) = 3</code>.
-	 * 
+	 *
 	 * @param value The value to compute the logarithm for.
 	 * @return The logarithm (rounded down) to the base of 2.
 	 * @throws ArithmeticException Thrown, if the given value is zero.
@@ -40,11 +40,11 @@ public final class MathUtils {
 
 		return 31 - Integer.numberOfLeadingZeros(value);
 	}
-	
+
 	/**
 	 * Computes the logarithm of the given value to the base of 2. This method throws an error,
 	 * if the given argument is not a power of 2.
-	 * 
+	 *
 	 * @param value The value to compute the logarithm for.
 	 * @return The logarithm to the base of 2.
 	 * @throws ArithmeticException Thrown, if the given value is zero.
@@ -59,25 +59,25 @@ public final class MathUtils {
 		}
 		return 31 - Integer.numberOfLeadingZeros(value);
 	}
-	
+
 	/**
 	 * Decrements the given number down to the closest power of two. If the argument is a
 	 * power of two, it remains unchanged.
-	 * 
+	 *
 	 * @param value The value to round down.
 	 * @return The closest value that is a power of two and less or equal than the given value.
 	 */
 	public static int roundDownToPowerOf2(int value) {
 		return Integer.highestOneBit(value);
 	}
-	
+
 	/**
 	 * Casts the given value to a 32 bit integer, if it can be safely done. If the cast would change the numeric
 	 * value, this method raises an exception.
-	 * <p>
-	 * This method is a protection in places where one expects to be able to safely case, but where unexpected
+	 *
+	 * <p>This method is a protection in places where one expects to be able to safely case, but where unexpected
 	 * situations could make the cast unsafe and would cause hidden problems that are hard to track down.
-	 * 
+	 *
 	 * @param value The value to be cast to an integer.
 	 * @return The given value as an integer.
 	 * @see Math#toIntExact(long)
@@ -106,7 +106,7 @@ public final class MathUtils {
 	 * The hash function has the <i>full avalanche</i> property, meaning that every bit of the value to be hashed
 	 * affects every bit of the hash value.
 	 *
-	 * It is crucial to use different hash functions to partition data across machines and the internal partitioning of
+	 * <p>It is crucial to use different hash functions to partition data across machines and the internal partitioning of
 	 * data structures. This hash function is intended for partitioning internally in data structures.
 	 *
 	 * @param code The integer to be hashed.
@@ -125,7 +125,7 @@ public final class MathUtils {
 	/**
 	 * This function hashes an integer value.
 	 *
-	 * It is crucial to use different hash functions to partition data across machines and the internal partitioning of
+	 * <p>It is crucial to use different hash functions to partition data across machines and the internal partitioning of
 	 * data structures. This hash function is intended for partitioning across machines.
 	 *
 	 * @param code The integer to be hashed.
@@ -154,7 +154,7 @@ public final class MathUtils {
 	}
 
 	/**
-	 * Round the given number to the next power of two
+	 * Round the given number to the next power of two.
 	 * @param x number to round
 	 * @return x rounded up to the next power of two
 	 */
@@ -198,7 +198,7 @@ public final class MathUtils {
 	}
 
 	// ============================================================================================
-	
+
 	/**
 	 * Prevent Instantiation through private constructor.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java
index 9273330..99b9644 100644
--- a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
@@ -23,7 +24,7 @@ import java.io.IOException;
 
 /**
  * A simple iterator interface. The key differences to the {@link java.util.Iterator} are
- * 
+ *
  * <ul>
  *   <li>It has two distinct <code>next()</code>, where one variant allows to pass an object that may
  *       be reused, if the type is mutable.</li>
@@ -31,23 +32,23 @@ import java.io.IOException;
  *       splitting it over two different functions such as <code>hasNext()</code> and <code>next()</code>
  *       </li>
  * </ul>
- * 
+ *
  * @param <E> The element type of the collection iterated over.
  */
 @Internal
 public interface MutableObjectIterator<E> {
-	
+
 	/**
 	 * Gets the next element from the collection. The contents of that next element is put into the
 	 * given reuse object, if the type is mutable.
-	 * 
+	 *
 	 * @param reuse The target object into which to place next element if E is mutable.
 	 * @return The filled object or <code>null</code> if the iterator is exhausted.
-	 * 
-	 * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the 
+	 *
+	 * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the
 	 *                     serialization / deserialization logic
 	 */
-	public E next(E reuse) throws IOException;
+	E next(E reuse) throws IOException;
 
 	/**
 	 * Gets the next element from the collection. The iterator implementation
@@ -58,5 +59,5 @@ public interface MutableObjectIterator<E> {
 	 * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the
 	 *                     serialization / deserialization logic
 	 */
-	public E next() throws IOException;
+	E next() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
index f56b452..447e8a9 100644
--- a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
@@ -19,8 +19,8 @@
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
-
 import org.apache.flink.configuration.IllegalConfigurationException;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import sun.net.util.IPAddressUtil;
@@ -38,18 +38,21 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 
+/**
+ * Utility for various network related tasks (such as finding free ports).
+ */
 @Internal
 public class NetUtils {
 
 	private static final Logger LOG = LoggerFactory.getLogger(NetUtils.class);
 
-	/** The wildcard address to listen on all interfaces (either 0.0.0.0 or ::) */
+	/** The wildcard address to listen on all interfaces (either 0.0.0.0 or ::). */
 	private static final String WILDCARD_ADDRESS = new InetSocketAddress(0).getAddress().getHostAddress();
-	
+
 	/**
 	 * Turn a fully qualified domain name (fqdn) into a hostname. If the fqdn has multiple subparts
 	 * (separated by a period '.'), it will take the first part. Otherwise it takes the entire fqdn.
-	 * 
+	 *
 	 * @param fqdn The fully qualified domain name.
 	 * @return The hostname.
 	 */
@@ -58,7 +61,7 @@ public class NetUtils {
 			throw new IllegalArgumentException("fqdn is null");
 		}
 		int dotPos = fqdn.indexOf('.');
-		if(dotPos == -1) {
+		if (dotPos == -1) {
 			return fqdn;
 		} else {
 			return fqdn.substring(0, dotPos);
@@ -68,31 +71,31 @@ public class NetUtils {
 	/**
 	 * Method to validate if the given String represents a hostname:port.
 	 *
-	 * Works also for ipv6.
+	 * <p>Works also for ipv6.
 	 *
-	 * See: http://stackoverflow.com/questions/2345063/java-common-way-to-validate-and-convert-hostport-to-inetsocketaddress
+	 * <p>See: http://stackoverflow.com/questions/2345063/java-common-way-to-validate-and-convert-hostport-to-inetsocketaddress
 	 *
 	 * @return URL object for accessing host and Port
 	 */
 	public static URL getCorrectHostnamePort(String hostPort) {
 		try {
-			URL u = new URL("http://"+hostPort);
-			if(u.getHost() == null) {
-				throw new IllegalArgumentException("The given host:port ('"+hostPort+"') doesn't contain a valid host");
+			URL u = new URL("http://" + hostPort);
+			if (u.getHost() == null) {
+				throw new IllegalArgumentException("The given host:port ('" + hostPort + "') doesn't contain a valid host");
 			}
-			if(u.getPort() == -1) {
-				throw new IllegalArgumentException("The given host:port ('"+hostPort+"') doesn't contain a valid port");
+			if (u.getPort() == -1) {
+				throw new IllegalArgumentException("The given host:port ('" + hostPort + "') doesn't contain a valid port");
 			}
 			return u;
 		} catch (MalformedURLException e) {
-			throw new IllegalArgumentException("The given host:port ('"+hostPort+"') is invalid", e);
+			throw new IllegalArgumentException("The given host:port ('" + hostPort + "') is invalid", e);
 		}
 	}
 
 	// ------------------------------------------------------------------------
 	//  Lookup of to free ports
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Find a non-occupied port.
 	 *
@@ -111,7 +114,7 @@ public class NetUtils {
 
 		throw new RuntimeException("Could not find a free permitted port on the machine.");
 	}
-	
+
 
 	// ------------------------------------------------------------------------
 	//  Encoding of IP addresses for URLs
@@ -168,7 +171,7 @@ public class NetUtils {
 	/**
 	 * Encodes an IP address properly as a URL string. This method makes sure that IPv6 addresses
 	 * have the proper formatting to be included in URLs.
-	 * 
+	 *
 	 * @param address The IP address to encode.
 	 * @return The proper URL string encoded IP address.
 	 */
@@ -202,7 +205,7 @@ public class NetUtils {
 	/**
 	 * Encodes an IP address and port to be included in URL. in particular, this method makes
 	 * sure that IPv6 addresses have the proper formatting to be included in URLs.
-	 * 
+	 *
 	 * @param address The socket address with the IP address and port.
 	 * @return The proper URL string encoded IP address and port.
 	 */
@@ -214,7 +217,7 @@ public class NetUtils {
 	}
 
 	/**
-	 * Normalizes and encodes a hostname and port to be included in URL. 
+	 * Normalizes and encodes a hostname and port to be included in URL.
 	 * In particular, this method makes sure that IPv6 address literals have the proper
 	 * formatting to be included in URLs.
 	 *
@@ -277,7 +280,7 @@ public class NetUtils {
 		// convert into text form
 		StringBuilder buf = new StringBuilder(40);
 		buf.append('[');
-		
+
 		boolean lastWasNumber = false;
 		for (int i = 0; i < hextets.length; i++) {
 			boolean thisIsNumber = hextets[i] >= 0;
@@ -296,11 +299,11 @@ public class NetUtils {
 		buf.append(']');
 		return buf.toString();
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Port range parsing
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Returns an iterator over available ports defined by the range definition.
 	 *
@@ -310,9 +313,9 @@ public class NetUtils {
 	 */
 	public static Iterator<Integer> getPortRangeFromString(String rangeDefinition) throws NumberFormatException {
 		final String[] ranges = rangeDefinition.trim().split(",");
-		
+
 		UnionIterator<Integer> iterators = new UnionIterator<>();
-		
+
 		for (String rawRange: ranges) {
 			Iterator<Integer> rangeIterator;
 			String range = rawRange.trim();
@@ -332,7 +335,7 @@ public class NetUtils {
 					throw new IllegalConfigurationException("Invalid port configuration. Port must be between 0" +
 						"and 65535, but was " + start + ".");
 				}
-				final int end = Integer.valueOf(range.substring(dashIdx+1, range.length()));
+				final int end = Integer.valueOf(range.substring(dashIdx + 1, range.length()));
 				if (end < 0 || end > 65535) {
 					throw new IllegalConfigurationException("Invalid port configuration. Port must be between 0" +
 						"and 65535, but was " + end + ".");
@@ -357,7 +360,7 @@ public class NetUtils {
 			}
 			iterators.add(rangeIterator);
 		}
-		
+
 		return iterators;
 	}
 
@@ -393,6 +396,10 @@ public class NetUtils {
 		return WILDCARD_ADDRESS;
 	}
 
+	/**
+	 * A factory for a local socket from port number.
+	 */
+	@FunctionalInterface
 	public interface SocketFactory {
 		ServerSocket createSocket(int port) throws IOException;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java b/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
index f9b02f3..b7377ec 100644
--- a/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
@@ -29,20 +29,20 @@ import java.util.NoSuchElementException;
  */
 @Public
 public class NumberSequenceIterator extends SplittableIterator<Long> {
-	
+
 	private static final long serialVersionUID = 1L;
 
-	/** The last number returned by the iterator */
+	/** The last number returned by the iterator. */
 	private final long to;
-	
-	/** The next number to be returned */
+
+	/** The next number to be returned. */
 	private long current;
-	
-	
+
+
 	/**
 	 * Creates a new splittable iterator, returning the range [from, to].
 	 * Both boundaries of the interval are inclusive.
-	 * 
+	 *
 	 * @param from The first number returned by the iterator.
 	 * @param to The last number returned by the iterator.
 	 */
@@ -50,15 +50,15 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 		if (from > to) {
 			throw new IllegalArgumentException("The 'to' value must not be smaller than the 'from' value.");
 		}
-		
+
 		this.current = from;
 		this.to = to;
 	}
-	
-	
+
+
 	/**
 	 * Internal constructor to allow for empty iterators.
-	 * 
+	 *
 	 * @param from The first number returned by the iterator.
 	 * @param to The last number returned by the iterator.
 	 * @param unused A dummy parameter to disambiguate the constructor.
@@ -67,11 +67,11 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 		this.current = from;
 		this.to = to;
 	}
-	
+
 	public long getCurrent() {
 		return this.current;
 	}
-	
+
 	public long getTo() {
 		return this.to;
 	}
@@ -100,15 +100,15 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 		if (numPartitions < 1) {
 			throw new IllegalArgumentException("The number of partitions must be at least 1.");
 		}
-		
+
 		if (numPartitions == 1) {
 			return new NumberSequenceIterator[] { new NumberSequenceIterator(current, to) };
 		}
-		
+
 		// here, numPartitions >= 2 !!!
-		
+
 		long elementsPerSplit;
-		
+
 		if (to - current + 1 >= 0) {
 			elementsPerSplit = (to - current + 1) / numPartitions;
 		}
@@ -118,10 +118,10 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 			// in most cases it holds that: current < 0 and to > 0, except for: to == 0 and current == Long.MIN_VALUE
 			// the later needs a special case
 			final long halfDiff; // must be positive
-			
+
 			if (current == Long.MIN_VALUE) {
 				// this means to >= 0
-				halfDiff = (Long.MAX_VALUE/2+1) + to/2;
+				halfDiff = (Long.MAX_VALUE / 2 + 1) + to / 2;
 			} else {
 				long posFrom = -current;
 				if (posFrom > to) {
@@ -132,35 +132,35 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 			}
 			elementsPerSplit = halfDiff / numPartitions * 2;
 		}
-		
+
 		if (elementsPerSplit < Long.MAX_VALUE) {
 			// figure out how many get one in addition
 			long numWithExtra = -(elementsPerSplit * numPartitions) + to - current + 1;
-			
+
 			// based on rounding errors, we may have lost one)
 			if (numWithExtra > numPartitions) {
 				elementsPerSplit++;
 				numWithExtra -= numPartitions;
-				
+
 				if (numWithExtra > numPartitions) {
 					throw new RuntimeException("Bug in splitting logic. To much rounding loss.");
 				}
 			}
-			
+
 			NumberSequenceIterator[] iters = new NumberSequenceIterator[numPartitions];
 			long curr = current;
 			int i = 0;
 			for (; i < numWithExtra; i++) {
 				long next = curr + elementsPerSplit + 1;
-				iters[i] = new NumberSequenceIterator(curr, next-1);
+				iters[i] = new NumberSequenceIterator(curr, next - 1);
 				curr = next;
 			}
 			for (; i < numPartitions; i++) {
 				long next = curr + elementsPerSplit;
-				iters[i] = new NumberSequenceIterator(curr, next-1, true);
+				iters[i] = new NumberSequenceIterator(curr, next - 1, true);
 				curr = next;
 			}
-			
+
 			return iters;
 		}
 		else {
@@ -168,22 +168,21 @@ public class NumberSequenceIterator extends SplittableIterator<Long> {
 			if (numPartitions != 2) {
 				throw new RuntimeException("Bug in splitting logic.");
 			}
-			
+
 			return new NumberSequenceIterator[] {
 				new NumberSequenceIterator(current, current + elementsPerSplit),
 				new NumberSequenceIterator(current + elementsPerSplit, to)
 			};
 		}
 	}
-	
 
 	@Override
 	public int getMaximumNumberOfSplits() {
-		if (to >= Integer.MAX_VALUE || current <= Integer.MIN_VALUE || to-current+1 >= Integer.MAX_VALUE) {
+		if (to >= Integer.MAX_VALUE || current <= Integer.MIN_VALUE || to - current + 1 >= Integer.MAX_VALUE) {
 			return Integer.MAX_VALUE;
 		}
 		else {
-			return (int) (to-current+1);
+			return (int) (to - current + 1);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/OperatingSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/OperatingSystem.java b/flink-core/src/main/java/org/apache/flink/util/OperatingSystem.java
index 7c477cc..efa7bf6 100644
--- a/flink-core/src/main/java/org/apache/flink/util/OperatingSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/util/OperatingSystem.java
@@ -25,29 +25,29 @@ import org.apache.flink.annotation.Internal;
  */
 @Internal
 public enum OperatingSystem {
-	
+
 	LINUX,
 	WINDOWS,
 	MAC_OS,
 	FREE_BSD,
 	SOLARIS,
 	UNKNOWN;
-	
+
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Gets the operating system that the JVM runs on from the java system properties.
 	 * this method returns <tt>UNKNOWN</tt>, if the operating system was not successfully determined.
-	 * 
+	 *
 	 * @return The enum constant for the operating system, or <tt>UNKNOWN</tt>, if it was not possible to determine.
 	 */
 	public static OperatingSystem getCurrentOperatingSystem() {
 		return os;
 	}
-	
+
 	/**
 	 * Checks whether the operating system this JVM runs on is Windows.
-	 * 
+	 *
 	 * @return <code>true</code> if the operating system this JVM runs on is
 	 *         Windows, <code>false</code> otherwise
 	 */
@@ -57,7 +57,7 @@ public enum OperatingSystem {
 
 	/**
 	 * Checks whether the operating system this JVM runs on is Linux.
-	 * 
+	 *
 	 * @return <code>true</code> if the operating system this JVM runs on is
 	 *         Linux, <code>false</code> otherwise
 	 */
@@ -67,7 +67,7 @@ public enum OperatingSystem {
 
 	/**
 	 * Checks whether the operating system this JVM runs on is Windows.
-	 * 
+	 *
 	 * @return <code>true</code> if the operating system this JVM runs on is
 	 *         Windows, <code>false</code> otherwise
 	 */
@@ -77,7 +77,7 @@ public enum OperatingSystem {
 
 	/**
 	 * Checks whether the operating system this JVM runs on is FreeBSD.
-	 * 
+	 *
 	 * @return <code>true</code> if the operating system this JVM runs on is
 	 *         FreeBSD, <code>false</code> otherwise
 	 */
@@ -94,21 +94,21 @@ public enum OperatingSystem {
 	public static boolean isSolaris() {
 		return getCurrentOperatingSystem() == SOLARIS;
 	}
-	
+
 	/**
 	 * The enum constant for the operating system.
 	 */
 	private static final OperatingSystem os = readOSFromSystemProperties();
-	
+
 	/**
 	 * Parses the operating system that the JVM runs on from the java system properties.
 	 * If the operating system was not successfully determined, this method returns {@code UNKNOWN}.
-	 * 
+	 *
 	 * @return The enum constant for the operating system, or {@code UNKNOWN}, if it was not possible to determine.
 	 */
 	private static OperatingSystem readOSFromSystemProperties() {
 		String osName = System.getProperty(OS_KEY);
-		
+
 		if (osName.startsWith(LINUX_OS_PREFIX)) {
 			return LINUX;
 		}
@@ -125,14 +125,14 @@ public enum OperatingSystem {
 		if (osNameLowerCase.contains(SOLARIS_OS_INFIX_1) || osNameLowerCase.contains(SOLARIS_OS_INFIX_2)) {
 			return SOLARIS;
 		}
-		
+
 		return UNKNOWN;
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	//  Constants to extract the OS type from the java environment 
+	//  Constants to extract the OS type from the java environment
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * The key to extract the operating system name from the system properties.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/OutputTag.java b/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
index 800c0b5..f132cb3 100644
--- a/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
+++ b/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.util;
 
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.Serializable;
+package org.apache.flink.util;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
 
 /**
  * An {@link OutputTag} is a typed and named tag to use for tagging side outputs

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
index e970c13..612a6e3 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
@@ -32,7 +32,7 @@ import javax.annotation.Nullable;
 
 /**
  * A collection of static utility methods to validate input.
- * 
+ *
  * <p>This class is modelled after Google Guava's Preconditions class, and partly takes code
  * from that class. We add this code to the Flink code base in order to reduce external
  * dependencies.
@@ -43,14 +43,14 @@ public final class Preconditions {
 	// ------------------------------------------------------------------------
 	//  Null checks
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Ensures that the given object reference is not null.
 	 * Upon violation, a {@code NullPointerException} with no message is thrown.
-	 * 
+	 *
 	 * @param reference The object reference
 	 * @return The object reference itself (generically typed).
-	 * 
+	 *
 	 * @throws NullPointerException Thrown, if the passed reference was null.
 	 */
 	public static <T> T checkNotNull(T reference) {
@@ -59,11 +59,11 @@ public final class Preconditions {
 		}
 		return reference;
 	}
-	
+
 	/**
 	 * Ensures that the given object reference is not null.
 	 * Upon violation, a {@code NullPointerException} with the given message is thrown.
-	 * 
+	 *
 	 * @param reference The object reference
 	 * @param errorMessage The message for the {@code NullPointerException} that is thrown if the check fails.
 	 * @return The object reference itself (generically typed).
@@ -80,7 +80,7 @@ public final class Preconditions {
 	/**
 	 * Ensures that the given object reference is not null.
 	 * Upon violation, a {@code NullPointerException} with the given message is thrown.
-	 * 
+	 *
 	 * <p>The error message is constructed from a template and an arguments array, after
 	 * a similar fashion as {@link String#format(String, Object...)}, but supporting only
 	 * {@code %s} as a placeholder.
@@ -91,7 +91,7 @@ public final class Preconditions {
 	 *                             {@code %s} placeholders with the error message arguments.
 	 * @param errorMessageArgs The arguments for the error message, to be inserted into the
 	 *                         message template for the {@code %s} placeholders.
-	 *                         
+	 *
 	 * @return The object reference itself (generically typed).
 	 *
 	 * @throws NullPointerException Thrown, if the passed reference was null.
@@ -99,7 +99,7 @@ public final class Preconditions {
 	public static <T> T checkNotNull(T reference,
 				@Nullable String errorMessageTemplate,
 				@Nullable Object... errorMessageArgs) {
-		
+
 		if (reference == null) {
 			throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs));
 		}
@@ -109,13 +109,13 @@ public final class Preconditions {
 	// ------------------------------------------------------------------------
 	//  Boolean Condition Checking (Argument)
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if
 	 * the condition is not met (evaluates to {@code false}).
 	 *
 	 * @param condition The condition to check
-	 *                     
+	 *
 	 * @throws IllegalArgumentException Thrown, if the condition is violated.
 	 */
 	public static void checkArgument(boolean condition) {
@@ -131,7 +131,7 @@ public final class Preconditions {
 	 *
 	 * @param condition The condition to check
 	 * @param errorMessage The message for the {@code IllegalArgumentException} that is thrown if the check fails.
-	 * 
+	 *
 	 * @throws IllegalArgumentException Thrown, if the condition is violated.
 	 */
 	public static void checkArgument(boolean condition, @Nullable Object errorMessage) {
@@ -150,13 +150,13 @@ public final class Preconditions {
 	 *                             {@code %s} placeholders with the error message arguments.
 	 * @param errorMessageArgs The arguments for the error message, to be inserted into the
 	 *                         message template for the {@code %s} placeholders.
-	 * 
+	 *
 	 * @throws IllegalArgumentException Thrown, if the condition is violated.
 	 */
 	public static void checkArgument(boolean condition,
 			@Nullable String errorMessageTemplate,
 			@Nullable Object... errorMessageArgs) {
-		
+
 		if (!condition) {
 			throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs));
 		}
@@ -165,7 +165,7 @@ public final class Preconditions {
 	// ------------------------------------------------------------------------
 	//  Boolean Condition Checking (State)
 	// ------------------------------------------------------------------------
-	
+
 	/**
 	 * Checks the given boolean condition, and throws an {@code IllegalStateException} if
 	 * the condition is not met (evaluates to {@code false}).
@@ -220,10 +220,10 @@ public final class Preconditions {
 
 	/**
 	 * Ensures that the given index is valid for an array, list or string of the given size.
-	 * 
+	 *
 	 * @param index index to check
 	 * @param size size of the array, list or string
-	 * 
+	 *
 	 * @throws IllegalArgumentException Thrown, if size is negative.
 	 * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size
      */
@@ -239,7 +239,7 @@ public final class Preconditions {
 	 *
 	 * @param index index to check
 	 * @param size size of the array, list or string
-	 * @param errorMessage The message for the {@code IndexOutOfBoundsException} that is thrown if the check fails.   
+	 * @param errorMessage The message for the {@code IndexOutOfBoundsException} that is thrown if the check fails.
 	 *
 	 * @throws IllegalArgumentException Thrown, if size is negative.
 	 * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size
@@ -258,13 +258,13 @@ public final class Preconditions {
 	/**
 	 * A simplified formatting method. Similar to {@link String#format(String, Object...)}, but
 	 * with lower overhead (only String parameters, no locale, no format validation).
-	 * 
+	 *
 	 * <p>This method is taken quasi verbatim from the Guava Preconditions class.
 	 */
 	private static String format(@Nullable String template, @Nullable Object... args) {
 		final int numArgs = args == null ? 0 : args.length;
 		template = String.valueOf(template); // null -> "null"
-		
+
 		// start substituting the arguments into the '%s' placeholders
 		StringBuilder builder = new StringBuilder(template.length() + 16 * numArgs);
 		int templateStart = 0;
@@ -296,6 +296,6 @@ public final class Preconditions {
 
 	// ------------------------------------------------------------------------
 
-	/** Private constructor to prevent instantiation */
+	/** Private constructor to prevent instantiation. */
 	private Preconditions() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/PropertiesUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/PropertiesUtil.java b/flink-core/src/main/java/org/apache/flink/util/PropertiesUtil.java
index 8650dcd..37d8a42 100644
--- a/flink-core/src/main/java/org/apache/flink/util/PropertiesUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/PropertiesUtil.java
@@ -34,7 +34,6 @@ public class PropertiesUtil {
 	 * @param key key in Properties
 	 * @param defaultValue default value if value is not set
 	 * @return default or value of key
-	 * @throws IllegalArgumentException
 	 */
 	public static int getInt(Properties config, String key, int defaultValue) {
 		String val = config.getProperty(key);
@@ -58,7 +57,6 @@ public class PropertiesUtil {
 	 * @param key key in Properties
 	 * @param defaultValue default value if value is not set
 	 * @return default or value of key
-	 * @throws IllegalArgumentException
 	 */
 	public static long getLong(Properties config, String key, long defaultValue) {
 		String val = config.getProperty(key);
@@ -82,12 +80,11 @@ public class PropertiesUtil {
 	 * @param key key in Properties
 	 * @param defaultValue default value if value is not set
 	 * @return default or value of key
-	 * @throws IllegalArgumentException
 	 */
 	public static long getLong(Properties config, String key, long defaultValue, Logger logger) {
 		try {
 			return getLong(config, key, defaultValue);
-		} catch(IllegalArgumentException iae) {
+		} catch (IllegalArgumentException iae) {
 			logger.warn(iae.getMessage());
 			return defaultValue;
 		}
@@ -112,7 +109,7 @@ public class PropertiesUtil {
 	}
 
 	// ------------------------------------------------------------------------
-	
-	/** Private default constructor to prevent instantiation */
+
+	/** Private default constructor to prevent instantiation. */
 	private PropertiesUtil() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
index 2883570..c212ca8 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
@@ -27,8 +26,13 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 
+/**
+ * Utility for reflection operations on classes and generic type parametrization.
+ */
 @Internal
+@SuppressWarnings("unused")
 public final class ReflectionUtil {
+
 	public static <T> T newInstance(Class<T> clazz) {
 		try {
 			return clazz.newInstance();
@@ -41,16 +45,17 @@ public final class ReflectionUtil {
 	public static <T> Class<T> getTemplateType(Class<?> clazz, int num) {
 		return (Class<T>) getSuperTemplateTypes(clazz)[num];
 	}
-	
+
 	@SuppressWarnings("unchecked")
 	public static <T> Class<T> getTemplateType(Class<?> clazz, Class<?> classWithParameter, int num) {
 		return (Class<T>) getSuperTemplateTypes(clazz)[num];
 	}
-	
+
 	public static <T> Class<T> getTemplateType1(Class<?> clazz) {
 		return getTemplateType(clazz, 0);
 	}
 
+	@SuppressWarnings("unchecked")
 	public static <T> Class<T> getTemplateType1(Type type) {
 		if (type instanceof ParameterizedType) {
 			return (Class<T>) getTemplateTypes((ParameterizedType) type)[0];
@@ -102,13 +107,13 @@ public final class ReflectionUtil {
 			clazz = clazz.getSuperclass();
 		}
 	}
-	
+
 	public static Class<?>[] getSuperTemplateTypes(Class<?> clazz, Class<?> searchedSuperClass) {
 		if (clazz == null || searchedSuperClass == null) {
 			throw new NullPointerException();
 		}
-		
-		Class<?> superClass = null;
+
+		Class<?> superClass;
 		do {
 			superClass = clazz.getSuperclass();
 			if (superClass == searchedSuperClass) {
@@ -116,11 +121,11 @@ public final class ReflectionUtil {
 			}
 		}
 		while ((clazz = superClass) != null);
-		
+
 		if (clazz == null) {
 			throw new IllegalArgumentException("The searched for superclass is not a superclass of the given class.");
 		}
-		
+
 		final Type type = clazz.getGenericSuperclass();
 		if (type instanceof ParameterizedType) {
 			return getTemplateTypes((ParameterizedType) type);
@@ -185,7 +190,7 @@ public final class ReflectionUtil {
 				templateTypeInfos[i] = getFullTemplateType(parameterizedType.getActualTypeArguments()[i]);
 			}
 
-			return new FullTypeInfo((Class<?>)parameterizedType.getRawType(), templateTypeInfos);
+			return new FullTypeInfo((Class<?>) parameterizedType.getRawType(), templateTypeInfos);
 		} else {
 			return new FullTypeInfo((Class<?>) type, null);
 		}
@@ -197,10 +202,10 @@ public final class ReflectionUtil {
 	 * describing the type.
 	 */
 	public static class FullTypeInfo {
+
 		private final Class<?> clazz;
 		private final FullTypeInfo[] templateTypeInfos;
 
-
 		public FullTypeInfo(Class<?> clazz, FullTypeInfo[] templateTypeInfos) {
 			this.clazz = Preconditions.checkNotNull(clazz);
 			this.templateTypeInfos = templateTypeInfos;

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/SerializableObject.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializableObject.java b/flink-core/src/main/java/org/apache/flink/util/SerializableObject.java
index ec01731..f6e4a97 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SerializableObject.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SerializableObject.java
@@ -23,6 +23,6 @@ package org.apache.flink.util;
  * in serializable classes.
  */
 public class SerializableObject implements java.io.Serializable {
-	
+
 	private static final long serialVersionUID = -7322636177391854669L;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
index 3cab55b..de6358c 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
@@ -31,22 +31,22 @@ import static java.util.Objects.requireNonNull;
  * Utility class for dealing with user-defined Throwable types that are serialized (for
  * example during RPC/Actor communication), but cannot be resolved with the default
  * class loader.
- * 
+ *
  * <p>This exception mimics the original exception with respect to message and stack trace,
  * and contains the original exception in serialized form. The original exception
  * can be re-obtained by supplying the appropriate class loader.
  */
 public class SerializedThrowable extends Exception implements Serializable {
-	
+
 	private static final long serialVersionUID = 7284183123441947635L;
-	
-	/** The original exception in serialized form */
+
+	/** The original exception in serialized form. */
 	private final byte[] serializedException;
-	
-	/** Name of the original error class */
+
+	/** Name of the original error class. */
 	private final String originalErrorClassName;
-	
-	/** The original stack trace, to be printed */
+
+	/** The original stack trace, to be printed. */
 	private final String fullStringifiedStackTrace;
 
 	/** The original exception, not transported via serialization,
@@ -57,11 +57,11 @@ public class SerializedThrowable extends Exception implements Serializable {
 
 	/**
 	 * Create a new SerializedThrowable.
-	 * 
+	 *
 	 * @param exception The exception to serialize.
 	 */
 	public SerializedThrowable(Throwable exception) {
-		this(exception, new HashSet<Throwable>());
+		this(exception, new HashSet<>());
 	}
 
 	/**
@@ -89,7 +89,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 				serialized = null;
 			}
 			this.serializedException = serialized;
-			this.cachedException = new WeakReference<Throwable>(exception);
+			this.cachedException = new WeakReference<>(exception);
 
 			// record the original exception's properties (name, stack prints)
 			this.originalErrorClassName = exception.getClass().getName();
@@ -103,7 +103,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 				initCause(null);
 			}
 			else {
-				// exception causes may by cyclic, so we truncate the cycle when we find it 
+				// exception causes may by cyclic, so we truncate the cycle when we find it
 				if (alreadySeen.add(exception)) {
 					// we are not in a cycle, yet
 					initCause(new SerializedThrowable(exception.getCause(), alreadySeen));
@@ -134,7 +134,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 		if (cached == null) {
 			try {
 				cached = InstantiationUtil.deserializeObject(serializedException, classloader);
-				cachedException = new WeakReference<Throwable>(cached);
+				cachedException = new WeakReference<>(cached);
 			}
 			catch (Throwable t) {
 				// something went wrong
@@ -166,13 +166,13 @@ public class SerializedThrowable extends Exception implements Serializable {
 		s.print(fullStringifiedStackTrace);
 		s.flush();
 	}
-	
+
 	@Override
 	public void printStackTrace(PrintWriter s) {
 		s.print(fullStringifiedStackTrace);
 		s.flush();
 	}
-	
+
 	@Override
 	public String toString() {
 		String message = getLocalizedMessage();
@@ -185,7 +185,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 
 	public static Throwable get(Throwable serThrowable, ClassLoader loader) {
 		if (serThrowable instanceof SerializedThrowable) {
-			return ((SerializedThrowable)serThrowable).deserializeError(loader);
+			return ((SerializedThrowable) serThrowable).deserializeError(loader);
 		} else {
 			return serThrowable;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java b/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
index d91c14e..32f9642 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
@@ -28,7 +28,7 @@ import java.util.Arrays;
  * in the system class loader. When those objects are deserialized without access to their
  * special class loader, the deserialization fails with a {@code ClassNotFoundException}.
  *
- * To work around that issue, the SerializedValue serialized data immediately into a byte array.
+ * <p>To work around that issue, the SerializedValue serialized data immediately into a byte array.
  * When send through RPC or another service that uses serialization, only the byte array is
  * transferred. The object is deserialized later (upon access) and requires the accessor to
  * provide the corresponding class loader.
@@ -40,7 +40,7 @@ public class SerializedValue<T> implements java.io.Serializable {
 
 	private static final long serialVersionUID = -3564011643393683761L;
 
-	/** The serialized data */
+	/** The serialized data. */
 	private final byte[] serializedData;
 
 	private SerializedValue(byte[] serializedData) {
@@ -68,7 +68,7 @@ public class SerializedValue<T> implements java.io.Serializable {
 	}
 
 	public static <T> SerializedValue<T> fromBytes(byte[] serializedData) {
-		return new SerializedValue<T>(serializedData);
+		return new SerializedValue<>(serializedData);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/SplittableIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SplittableIterator.java b/flink-core/src/main/java/org/apache/flink/util/SplittableIterator.java
index bb7f9af..993c521 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SplittableIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SplittableIterator.java
@@ -18,14 +18,15 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.annotation.Public;
+
 import java.io.Serializable;
 import java.util.Iterator;
-import org.apache.flink.annotation.Public;
 
 /**
  * Abstract base class for iterators that can split themselves into multiple disjoint
  * iterators. The union of these iterators returns the original iterator values.
- * 
+ *
  * @param <T> The type of elements returned by the iterator.
  */
 @Public
@@ -36,7 +37,7 @@ public abstract class SplittableIterator<T> implements Iterator<T>, Serializable
 	/**
 	 * Splits this iterator into a number disjoint iterators.
 	 * The union of these iterators returns the original iterator values.
-	 * 
+	 *
 	 * @param numPartitions The number of iterators to split into.
 	 * @return An array with the split iterators.
 	 */
@@ -45,7 +46,7 @@ public abstract class SplittableIterator<T> implements Iterator<T>, Serializable
 	/**
 	 * Splits this iterator into <i>n</i> partitions and returns the <i>i-th</i> partition
 	 * out of those.
-	 * 
+	 *
 	 * @param num The partition to return (<i>i</i>).
 	 * @param numPartitions The number of partitions to split into (<i>n</i>).
 	 * @return The iterator for the partition.
@@ -60,7 +61,7 @@ public abstract class SplittableIterator<T> implements Iterator<T>, Serializable
 
 	/**
 	 * The maximum number of splits into which this iterator can be split up.
-	 * 
+	 *
 	 * @return The maximum number of splits into which this iterator can be split up.
 	 */
 	public abstract int getMaximumNumberOfSplits();

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/StateMigrationException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StateMigrationException.java b/flink-core/src/main/java/org/apache/flink/util/StateMigrationException.java
index 1667ff5..00e0e73 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StateMigrationException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StateMigrationException.java
@@ -19,7 +19,7 @@
 package org.apache.flink.util;
 
 /**
- * Base class for state migration related exceptions
+ * Base class for state migration related exceptions.
  */
 public class StateMigrationException extends FlinkException {
 	private static final long serialVersionUID = 8268516412747670839L;

http://git-wip-us.apache.org/repos/asf/flink/blob/212ee3d4/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java b/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
index fe03fa9..7ec8df5 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
@@ -29,7 +29,7 @@ public class StringBasedID implements Serializable {
 	private static final long serialVersionUID = 1L;
 
 	/**
-	 * Uses a String as internal representation
+	 * Uses a String as internal representation.
 	 */
 	private final String keyString;
 


[02/17] flink git commit: [FLINK-5823] [checkpoints] State backends now also handle the checkpoint metadata

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 293675c..98b6647 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
@@ -25,11 +25,12 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
-import org.mockito.Mockito;
 
 import java.io.File;
 import java.util.Collections;
@@ -63,7 +64,7 @@ public class CompletedCheckpointTest {
 				new JobID(), 0, 0, 1,
 				taskStates,
 				Collections.<MasterState>emptyList(),
-				CheckpointProperties.forStandardCheckpoint(),
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 				new FileStateHandle(new Path(file.toURI()), file.length()),
 				file.getAbsolutePath());
 
@@ -81,16 +82,18 @@ public class CompletedCheckpointTest {
 		Map<OperatorID, OperatorState> operatorStates = new HashMap<>();
 		operatorStates.put(new OperatorID(), state);
 
+		StreamStateHandle metadataHandle = mock(StreamStateHandle.class);
+
 		boolean discardSubsumed = true;
-		CheckpointProperties props = new CheckpointProperties(false, false, false, discardSubsumed, true, true, true, true);
+		CheckpointProperties props = new CheckpointProperties(false, false, discardSubsumed, true, true, true, true);
 		
 		CompletedCheckpoint checkpoint = new CompletedCheckpoint(
 				new JobID(), 0, 0, 1,
 				operatorStates,
 				Collections.<MasterState>emptyList(),
 				props,
-				null,
-				null);
+				metadataHandle,
+				"some/mock/pointer");
 
 		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 		checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry);
@@ -100,6 +103,7 @@ public class CompletedCheckpointTest {
 		checkpoint.discardOnSubsume();
 
 		verify(state, times(1)).discardState();
+		verify(metadataHandle).discardState();
 	}
 
 	/**
@@ -107,49 +111,48 @@ public class CompletedCheckpointTest {
 	 */
 	@Test
 	public void testCleanUpOnShutdown() throws Exception {
-		File file = tmpFolder.newFile();
-		String externalPath = file.getAbsolutePath();
-
 		JobStatus[] terminalStates = new JobStatus[] {
 				JobStatus.FINISHED, JobStatus.CANCELED, JobStatus.FAILED, JobStatus.SUSPENDED
 		};
 
-		OperatorState state = mock(OperatorState.class);
-		Map<OperatorID, OperatorState> operatorStates = new HashMap<>();
-		operatorStates.put(new OperatorID(), state);
-
 		for (JobStatus status : terminalStates) {
-			Mockito.reset(state);
+
+			OperatorState state = mock(OperatorState.class);
+			Map<OperatorID, OperatorState> operatorStates = new HashMap<>();
+			operatorStates.put(new OperatorID(), state);
+
+			StreamStateHandle metadataHandle = mock(StreamStateHandle.class);
 
 			// Keep
-			CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false);
+			CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false);
 			CompletedCheckpoint checkpoint = new CompletedCheckpoint(
 					new JobID(), 0, 0, 1,
 					new HashMap<>(operatorStates),
 					Collections.<MasterState>emptyList(),
 					props,
-					new FileStateHandle(new Path(file.toURI()), file.length()),
-					externalPath);
+					metadataHandle,
+					"mock://some/pointer");
 
 			SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 			checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry);
 
 			checkpoint.discardOnShutdown(status);
 			verify(state, times(0)).discardState();
-			assertEquals(true, file.exists());
+			verify(metadataHandle, times(0)).discardState();
 
 			// Discard
-			props = new CheckpointProperties(false, false, false, true, true, true, true, true);
+			props = new CheckpointProperties(false, false, true, true, true, true, true);
 			checkpoint = new CompletedCheckpoint(
 					new JobID(), 0, 0, 1,
 					new HashMap<>(operatorStates),
 					Collections.<MasterState>emptyList(),
 					props,
-					null,
-					null);
+					metadataHandle,
+					"pointer");
 
 			checkpoint.discardOnShutdown(status);
 			verify(state, times(1)).discardState();
+			verify(metadataHandle, times(1)).discardState();
 		}
 	}
 
@@ -169,9 +172,9 @@ public class CompletedCheckpointTest {
 			1,
 			new HashMap<>(operatorStates),
 			Collections.<MasterState>emptyList(),
-			CheckpointProperties.forStandardCheckpoint(),
-			null,
-			null);
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			mock(StreamStateHandle.class),
+			"pointer");
 
 		CompletedCheckpointStats.DiscardCallback callback = mock(CompletedCheckpointStats.DiscardCallback.class);
 		completed.setDiscardCallback(callback);
@@ -192,7 +195,7 @@ public class CompletedCheckpointTest {
 		CompletedCheckpointStats completed = new CompletedCheckpointStats(
 			123123123L,
 			10123L,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			1337,
 			taskStats,
 			1337,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 7d6c7b5..1d44444 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
@@ -29,15 +29,14 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 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.FailingBlockingInvokable;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
 import scala.concurrent.Await;
@@ -80,7 +79,7 @@ public class CoordinatorShutdownTest extends TestLogger {
 						60000,
 						0L,
 						Integer.MAX_VALUE,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null));
 			
@@ -149,7 +148,7 @@ public class CoordinatorShutdownTest extends TestLogger {
 						60000,
 						0L,
 						Integer.MAX_VALUE,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null));
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 6a84a11..a53d6d2 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
@@ -27,7 +27,6 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
@@ -95,14 +94,13 @@ public class ExecutionGraphCheckpointCoordinatorTest {
 				100,
 				100,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				Collections.emptyList(),
 				Collections.emptyList(),
 				Collections.emptyList(),
 				Collections.emptyList(),
 				counter,
 				store,
-				null,
 				new MemoryStateBackend(),
 				CheckpointStatsTrackerTest.createTestTracker());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStatsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStatsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStatsTest.java
index f1a56be..7668f62 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStatsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStatsTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+
 import org.junit.Test;
 
 import java.io.NotSerializableException;
@@ -47,7 +48,7 @@ public class FailedCheckpointStatsTest {
 		FailedCheckpointStats failed = new FailedCheckpointStats(
 			0,
 			triggerTimestamp,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			1,
 			taskStats,
 			0,
@@ -73,7 +74,7 @@ public class FailedCheckpointStatsTest {
 		FailedCheckpointStats failed = new FailedCheckpointStats(
 			123123123L,
 			triggerTimestamp,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			1337,
 			taskStats,
 			3,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStatsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStatsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStatsTest.java
index 6c5e8fd..73db317 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStatsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStatsTest.java
@@ -42,7 +42,7 @@ public class PendingCheckpointStatsTest {
 	public void testReportSubtaskStats() throws Exception {
 		long checkpointId = Integer.MAX_VALUE + 1222L;
 		long triggerTimestamp = Integer.MAX_VALUE - 1239L;
-		CheckpointProperties props = CheckpointProperties.forStandardCheckpoint();
+		CheckpointProperties props = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION);
 		TaskStateStats task1 = new TaskStateStats(new JobVertexID(), 3);
 		TaskStateStats task2 = new TaskStateStats(new JobVertexID(), 4);
 		int totalSubtaskCount = task1.getNumberOfSubtasks() + task2.getNumberOfSubtasks();
@@ -138,7 +138,7 @@ public class PendingCheckpointStatsTest {
 		PendingCheckpointStats pending = new PendingCheckpointStats(
 			0,
 			1,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			task1.getNumberOfSubtasks() + task2.getNumberOfSubtasks(),
 			taskStats,
 			callback);
@@ -199,7 +199,7 @@ public class PendingCheckpointStatsTest {
 		PendingCheckpointStats pending = new PendingCheckpointStats(
 			0,
 			triggerTimestamp,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			task1.getNumberOfSubtasks() + task2.getNumberOfSubtasks(),
 			taskStats,
 			callback);
@@ -251,7 +251,7 @@ public class PendingCheckpointStatsTest {
 		PendingCheckpointStats pending = new PendingCheckpointStats(
 			123123123L,
 			10123L,
-			CheckpointProperties.forStandardCheckpoint(),
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
 			1337,
 			taskStats,
 			mock(CheckpointStatsTracker.PendingCheckpointStatsCallback.class));

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 bf79457..7b6992b 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
@@ -19,6 +19,8 @@
 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.fs.local.LocalFileSystem;
 import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -26,13 +28,15 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation;
+
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.mockito.Mockito;
 
-import java.io.File;
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayDeque;
 import java.util.Collections;
@@ -43,7 +47,6 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ScheduledFuture;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -55,6 +58,9 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.when;
 
+/**
+ * Tests for the {@link PendingCheckpoint}.
+ */
 public class PendingCheckpointTest {
 
 	private static final Map<ExecutionAttemptID, ExecutionVertex> ACK_TASKS = new HashMap<>();
@@ -63,7 +69,7 @@ public class PendingCheckpointTest {
 	static {
 		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
 		when(jobVertex.getOperatorIDs()).thenReturn(Collections.singletonList(new OperatorID()));
-		
+
 		ExecutionVertex vertex = mock(ExecutionVertex.class);
 		when(vertex.getMaxParallelism()).thenReturn(128);
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(1);
@@ -80,8 +86,8 @@ public class PendingCheckpointTest {
 	@Test
 	public void testCanBeSubsumed() throws Exception {
 		// Forced checkpoints cannot be subsumed
-		CheckpointProperties forced = new CheckpointProperties(true, true, false, false, false, false, false, false);
-		PendingCheckpoint pending = createPendingCheckpoint(forced, "ignored");
+		CheckpointProperties forced = new CheckpointProperties(true, true, false, false, false, false, false);
+		PendingCheckpoint pending = createPendingCheckpoint(forced);
 		assertFalse(pending.canBeSubsumed());
 
 		try {
@@ -92,48 +98,21 @@ public class PendingCheckpointTest {
 		}
 
 		// Non-forced checkpoints can be subsumed
-		CheckpointProperties subsumed = new CheckpointProperties(false, true, false, false, false, false, false, false);
-		pending = createPendingCheckpoint(subsumed, "ignored");
+		CheckpointProperties subsumed = new CheckpointProperties(false, true, false, false, false, false, false);
+		pending = createPendingCheckpoint(subsumed);
 		assertTrue(pending.canBeSubsumed());
 	}
 
 	/**
-	 * Tests that the persist checkpoint property is respected by the pending
-	 * checkpoint when finalizing.
-	 */
-	@Test
-	public void testPersistExternally() throws Exception {
-		File tmp = tmpFolder.newFolder();
-
-		// Persisted checkpoint
-		CheckpointProperties persisted = new CheckpointProperties(false, true, false, false, false, false, false, false);
-
-		PendingCheckpoint pending = createPendingCheckpoint(persisted, tmp.getAbsolutePath());
-		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
-		assertEquals(0, tmp.listFiles().length);
-		pending.finalizeCheckpointExternalized();
-		assertEquals(1, tmp.listFiles().length);
-
-		// Ephemeral checkpoint
-		CheckpointProperties ephemeral = new CheckpointProperties(false, false, false, true, true, true, true, true);
-		pending = createPendingCheckpoint(ephemeral, null);
-		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
-
-		assertEquals(1, tmp.listFiles().length);
-		pending.finalizeCheckpointNonExternalized();
-		assertEquals(1, tmp.listFiles().length);
-	}
-
-	/**
 	 * Tests that the completion future is succeeded on finalize and failed on
 	 * abort and failures during finalize.
 	 */
 	@Test
 	public void testCompletionFuture() throws Exception {
-		CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false);
+		CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false);
 
 		// Abort declined
-		PendingCheckpoint pending = createPendingCheckpoint(props, "ignored");
+		PendingCheckpoint pending = createPendingCheckpoint(props);
 		CompletableFuture<CompletedCheckpoint> future = pending.getCompletionFuture();
 
 		assertFalse(future.isDone());
@@ -141,7 +120,7 @@ public class PendingCheckpointTest {
 		assertTrue(future.isDone());
 
 		// Abort expired
-		pending = createPendingCheckpoint(props, "ignored");
+		pending = createPendingCheckpoint(props);
 		future = pending.getCompletionFuture();
 
 		assertFalse(future.isDone());
@@ -149,7 +128,7 @@ public class PendingCheckpointTest {
 		assertTrue(future.isDone());
 
 		// Abort subsumed
-		pending = createPendingCheckpoint(props, "ignored");
+		pending = createPendingCheckpoint(props);
 		future = pending.getCompletionFuture();
 
 		assertFalse(future.isDone());
@@ -157,29 +136,22 @@ public class PendingCheckpointTest {
 		assertTrue(future.isDone());
 
 		// Finalize (all ACK'd)
-		String target = tmpFolder.newFolder().getAbsolutePath();
-		pending = createPendingCheckpoint(props, target);
+		pending = createPendingCheckpoint(props);
 		future = pending.getCompletionFuture();
 
 		assertFalse(future.isDone());
 		pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
 		assertTrue(pending.isFullyAcknowledged());
-		pending.finalizeCheckpointExternalized();
+		pending.finalizeCheckpoint();
 		assertTrue(future.isDone());
 
 		// Finalize (missing ACKs)
-		pending = createPendingCheckpoint(props, "ignored");
+		pending = createPendingCheckpoint(props);
 		future = pending.getCompletionFuture();
 
 		assertFalse(future.isDone());
 		try {
-			pending.finalizeCheckpointNonExternalized();
-			fail("Did not throw expected Exception");
-		} catch (IllegalStateException ignored) {
-			// Expected
-		}
-		try {
-			pending.finalizeCheckpointExternalized();
+			pending.finalizeCheckpoint();
 			fail("Did not throw expected Exception");
 		} catch (IllegalStateException ignored) {
 			// Expected
@@ -192,16 +164,14 @@ public class PendingCheckpointTest {
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testAbortDiscardsState() throws Exception {
-		CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false);
+		CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false);
 		QueueExecutor executor = new QueueExecutor();
 
 		OperatorState state = mock(OperatorState.class);
 		doNothing().when(state).registerSharedStates(any(SharedStateRegistry.class));
 
-		String targetDir = tmpFolder.newFolder().getAbsolutePath();
-
 		// Abort declined
-		PendingCheckpoint pending = createPendingCheckpoint(props, targetDir, executor);
+		PendingCheckpoint pending = createPendingCheckpoint(props, executor);
 		setTaskState(pending, state);
 
 		pending.abortDeclined();
@@ -212,7 +182,7 @@ public class PendingCheckpointTest {
 		// Abort error
 		Mockito.reset(state);
 
-		pending = createPendingCheckpoint(props, targetDir, executor);
+		pending = createPendingCheckpoint(props, executor);
 		setTaskState(pending, state);
 
 		pending.abortError(new Exception("Expected Test Exception"));
@@ -223,7 +193,7 @@ public class PendingCheckpointTest {
 		// Abort expired
 		Mockito.reset(state);
 
-		pending = createPendingCheckpoint(props, targetDir, executor);
+		pending = createPendingCheckpoint(props, executor);
 		setTaskState(pending, state);
 
 		pending.abortExpired();
@@ -234,7 +204,7 @@ public class PendingCheckpointTest {
 		// Abort subsumed
 		Mockito.reset(state);
 
-		pending = createPendingCheckpoint(props, targetDir, executor);
+		pending = createPendingCheckpoint(props, executor);
 		setTaskState(pending, state);
 
 		pending.abortSubsumed();
@@ -251,20 +221,22 @@ public class PendingCheckpointTest {
 		{
 			// Complete successfully
 			PendingCheckpointStats callback = mock(PendingCheckpointStats.class);
-			PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+			PendingCheckpoint pending = createPendingCheckpoint(
+					CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 			pending.setStatsCallback(callback);
 
 			pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics());
 			verify(callback, times(1)).reportSubtaskStats(any(JobVertexID.class), any(SubtaskStateStats.class));
 
-			pending.finalizeCheckpointNonExternalized();
+			pending.finalizeCheckpoint();
 			verify(callback, times(1)).reportCompletedCheckpoint(any(String.class));
 		}
 
 		{
 			// Fail subsumed
 			PendingCheckpointStats callback = mock(PendingCheckpointStats.class);
-			PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+			PendingCheckpoint pending = createPendingCheckpoint(
+					CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 			pending.setStatsCallback(callback);
 
 			pending.abortSubsumed();
@@ -274,7 +246,8 @@ public class PendingCheckpointTest {
 		{
 			// Fail subsumed
 			PendingCheckpointStats callback = mock(PendingCheckpointStats.class);
-			PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+			PendingCheckpoint pending = createPendingCheckpoint(
+					CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 			pending.setStatsCallback(callback);
 
 			pending.abortDeclined();
@@ -284,7 +257,8 @@ public class PendingCheckpointTest {
 		{
 			// Fail subsumed
 			PendingCheckpointStats callback = mock(PendingCheckpointStats.class);
-			PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+			PendingCheckpoint pending = createPendingCheckpoint(
+					CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 			pending.setStatsCallback(callback);
 
 			pending.abortError(new Exception("Expected test error"));
@@ -294,7 +268,8 @@ public class PendingCheckpointTest {
 		{
 			// Fail subsumed
 			PendingCheckpointStats callback = mock(PendingCheckpointStats.class);
-			PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+			PendingCheckpoint pending = createPendingCheckpoint(
+					CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 			pending.setStatsCallback(callback);
 
 			pending.abortExpired();
@@ -303,41 +278,43 @@ public class PendingCheckpointTest {
 	}
 
 	/**
-	 * FLINK-5985
-	 * <p>
-	 * Ensures that subtasks that acknowledge their state as 'null' are considered stateless. This means that they
+	 * FLINK-5985.
+	 *
+	 * <p>Ensures that subtasks that acknowledge their state as 'null' are considered stateless. This means that they
 	 * should not appear in the task states map of the checkpoint.
 	 */
 	@Test
 	public void testNullSubtaskStateLeadsToStatelessTask() throws Exception {
-		PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+		PendingCheckpoint pending = createPendingCheckpoint(
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		pending.acknowledgeTask(ATTEMPT_ID, null, mock(CheckpointMetrics.class));
 		Assert.assertTrue(pending.getOperatorStates().isEmpty());
 	}
 
 	/**
-	 * FLINK-5985
-	 * <p>
-	 * This tests checks the inverse of {@link #testNullSubtaskStateLeadsToStatelessTask()}. We want to test that
+	 * FLINK-5985.
+	 *
+	 * <p>This tests checks the inverse of {@link #testNullSubtaskStateLeadsToStatelessTask()}. We want to test that
 	 * for subtasks that acknowledge some state are given an entry in the task states of the checkpoint.
 	 */
 	@Test
 	public void testNonNullSubtaskStateLeadsToStatefulTask() throws Exception {
-		PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null);
+		PendingCheckpoint pending = createPendingCheckpoint(
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		pending.acknowledgeTask(ATTEMPT_ID, mock(TaskStateSnapshot.class), mock(CheckpointMetrics.class));
 		Assert.assertFalse(pending.getOperatorStates().isEmpty());
 	}
 
 	@Test
-	public void testSetCanceller() {
-		final CheckpointProperties props = new CheckpointProperties(false, false, false, true, true, true, true, true);
+	public void testSetCanceller() throws Exception {
+		final CheckpointProperties props = new CheckpointProperties(false, false, true, true, true, true, true);
 
-		PendingCheckpoint aborted = createPendingCheckpoint(props, null);
+		PendingCheckpoint aborted = createPendingCheckpoint(props);
 		aborted.abortDeclined();
 		assertTrue(aborted.isDiscarded());
 		assertFalse(aborted.setCancellerHandle(mock(ScheduledFuture.class)));
 
-		PendingCheckpoint pending = createPendingCheckpoint(props, null);
+		PendingCheckpoint pending = createPendingCheckpoint(props);
 		ScheduledFuture<?> canceller = mock(ScheduledFuture.class);
 
 		assertTrue(pending.setCancellerHandle(canceller));
@@ -347,23 +324,25 @@ public class PendingCheckpointTest {
 
 	// ------------------------------------------------------------------------
 
-	private static PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, String targetDirectory) {
-		return createPendingCheckpoint(props, targetDirectory, Executors.directExecutor());
+	private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props) throws IOException {
+		return createPendingCheckpoint(props, Executors.directExecutor());
 	}
 
-	private static PendingCheckpoint createPendingCheckpoint(
-			CheckpointProperties props,
-			String targetDirectory,
-			Executor executor) {
+	private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Executor executor) throws IOException {
+
+		final Path checkpointDir = new Path(tmpFolder.newFolder().toURI());
+		final FsCheckpointStorageLocation location = new FsCheckpointStorageLocation(
+				LocalFileSystem.getSharedInstance(), checkpointDir, checkpointDir, checkpointDir);
+
+		final Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(ACK_TASKS);
 
-		Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(ACK_TASKS);
 		return new PendingCheckpoint(
 			new JobID(),
 			0,
 			1,
 			ackTasks,
 			props,
-			targetDirectory,
+			location,
 			executor);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java
index d43283d..85b1516 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java
@@ -31,7 +31,7 @@ public class RestoredCheckpointStatsTest {
 	public void testSimpleAccess() throws Exception {
 		long checkpointId = Integer.MAX_VALUE + 1L;
 		long triggerTimestamp = Integer.MAX_VALUE + 1L;
-		CheckpointProperties props = new CheckpointProperties(true, true, false, false, false, true, false, true);
+		CheckpointProperties props = new CheckpointProperties(true, true, false, false, true, false, true);
 		long restoreTimestamp = Integer.MAX_VALUE + 1L;
 		String externalPath = "external-path";
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java
index 7c19b19..a167130 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.checkpoint;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.state.EmptyStreamStateHandle;
 import org.apache.flink.runtime.state.RetrievableStateHandle;
 import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper;
 import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
@@ -32,8 +33,10 @@ import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.framework.api.ErrorListenerPathable;
 import org.apache.curator.utils.EnsurePath;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
+
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -92,8 +95,9 @@ public class ZooKeeperCompletedCheckpointStoreTest extends TestLogger {
 			1L,
 			new HashMap<>(),
 			null,
-			CheckpointProperties.forStandardCheckpoint(),
-			null, null);
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			new EmptyStreamStateHandle(),
+				"<pointer>");
 
 		final CompletedCheckpoint completedCheckpoint2 = new CompletedCheckpoint(
 			new JobID(),
@@ -102,8 +106,9 @@ public class ZooKeeperCompletedCheckpointStoreTest extends TestLogger {
 			2L,
 			new HashMap<>(),
 			null,
-			CheckpointProperties.forStandardCheckpoint(),
-			null, null);
+			CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
+			new EmptyStreamStateHandle(),
+			"<pointer");
 
 		final Collection<Long> expectedCheckpointIds = new HashSet<>(2);
 		expectedCheckpointIds.add(1L);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooksTest.java
index 3498a41..ccc8afe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooksTest.java
@@ -21,9 +21,11 @@ package org.apache.flink.runtime.checkpoint.hooks;
 import org.apache.flink.core.io.SimpleVersionedSerializer;
 import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
 import javax.annotation.Nullable;
+
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java
deleted file mode 100644
index a461569..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.checkpoint.savepoint;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
-import org.apache.flink.runtime.checkpoint.MasterState;
-import org.apache.flink.runtime.checkpoint.OperatorState;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.state.OperatorStateHandle;
-import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class SavepointLoaderTest {
-
-	@Rule
-	public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-	/**
-	 * Tests loading and validation of savepoints with correct setup,
-	 * parallelism mismatch, and a missing task.
-	 */
-	@Test
-	public void testLoadAndValidateSavepoint() throws Exception {
-		File tmp = tmpFolder.newFolder();
-
-		int parallelism = 128128;
-		long checkpointId = Integer.MAX_VALUE + 123123L;
-		JobVertexID jobVertexID = new JobVertexID();
-		OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID);
-
-		OperatorSubtaskState subtaskState = new OperatorSubtaskState(
-			new OperatorStateHandle(Collections.emptyMap(), new ByteStreamStateHandle("testHandler", new byte[0])),
-			null,
-			null,
-			null);
-
-		OperatorState state = new OperatorState(operatorID, parallelism, parallelism);
-		state.putState(0, subtaskState);
-
-		Map<OperatorID, OperatorState> taskStates = new HashMap<>();
-		taskStates.put(operatorID, state);
-
-		JobID jobId = new JobID();
-
-		// Store savepoint
-		SavepointV2 savepoint = new SavepointV2(checkpointId, taskStates.values(), Collections.<MasterState>emptyList());
-		String path = SavepointStore.storeSavepoint(tmp.getAbsolutePath(), savepoint);
-
-		ExecutionJobVertex vertex = mock(ExecutionJobVertex.class);
-		when(vertex.getParallelism()).thenReturn(parallelism);
-		when(vertex.getMaxParallelism()).thenReturn(parallelism);
-		when(vertex.getOperatorIDs()).thenReturn(Collections.singletonList(operatorID));
-
-		Map<JobVertexID, ExecutionJobVertex> tasks = new HashMap<>();
-		tasks.put(jobVertexID, vertex);
-
-		ClassLoader ucl = Thread.currentThread().getContextClassLoader();
-
-		// 1) Load and validate: everything correct
-		CompletedCheckpoint loaded = SavepointLoader.loadAndValidateSavepoint(jobId, tasks, path, ucl, false);
-
-		assertEquals(jobId, loaded.getJobId());
-		assertEquals(checkpointId, loaded.getCheckpointID());
-
-		// 2) Load and validate: max parallelism mismatch
-		when(vertex.getMaxParallelism()).thenReturn(222);
-		when(vertex.isMaxParallelismConfigured()).thenReturn(true);
-
-		try {
-			SavepointLoader.loadAndValidateSavepoint(jobId, tasks, path, ucl, false);
-			fail("Did not throw expected Exception");
-		} catch (IllegalStateException expected) {
-			assertTrue(expected.getMessage().contains("Max parallelism mismatch"));
-		}
-
-		// 3) Load and validate: missing vertex
-		assertNotNull(tasks.remove(jobVertexID));
-
-		try {
-			SavepointLoader.loadAndValidateSavepoint(jobId, tasks, path, ucl, false);
-			fail("Did not throw expected Exception");
-		} catch (IllegalStateException expected) {
-			assertTrue(expected.getMessage().contains("allowNonRestoredState"));
-		}
-
-		// 4) Load and validate: ignore missing vertex
-		SavepointLoader.loadAndValidateSavepoint(jobId, tasks, path, ucl, true);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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
deleted file mode 100644
index 0444936..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStoreTest.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.checkpoint.savepoint;
-
-import java.io.File;
-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.MasterState;
-import org.apache.flink.runtime.checkpoint.OperatorState;
-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;
-import org.mockito.Matchers;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-
-public class SavepointStoreTest {
-
-	@Rule
-	public TemporaryFolder tmp = new TemporaryFolder();
-
-	/**
-	 * Tests a store-load-dispose sequence.
-	 */
-	@Test
-	public void testStoreLoadDispose() throws Exception {
-		String root = tmp.getRoot().getAbsolutePath();
-		File rootFile = new File(root);
-
-		File[] list = rootFile.listFiles();
-
-		assertNotNull(list);
-		assertEquals(0, list.length);
-
-		// Store
-		String savepointDirectory = SavepointStore.createSavepointDirectory(root, new JobID());
-		SavepointV2 stored = new SavepointV2(
-			1929292,
-			CheckpointTestUtils.createOperatorStates(4, 24),
-			Collections.<MasterState>emptyList());
-		String path = SavepointStore.storeSavepoint(savepointDirectory, stored);
-
-		list = rootFile.listFiles();
-		assertNotNull(list);
-		assertEquals(1, list.length);
-
-		// Load
-		Savepoint loaded = SavepointStore.loadSavepoint(path, Thread.currentThread().getContextClassLoader());
-
-		assertEquals(stored.getCheckpointId(), loaded.getCheckpointId());
-		assertEquals(stored.getOperatorStates(), loaded.getOperatorStates());
-		assertEquals(stored.getMasterStates(), loaded.getMasterStates());
-
-		loaded.dispose();
-
-		// Dispose
-		SavepointStore.deleteSavepointDirectory(path);
-
-		list = rootFile.listFiles();
-		assertNotNull(list);
-		assertEquals(0, list.length);
-	}
-
-	/**
-	 * Tests loading with unexpected magic number.
-	 */
-	@Test
-	public void testUnexpectedSavepoint() throws Exception {
-		// Random file
-		Path filePath = new Path(tmp.getRoot().getPath(), UUID.randomUUID().toString());
-		FSDataOutputStream fdos = FileSystem.get(filePath.toUri()).create(filePath, FileSystem.WriteMode.NO_OVERWRITE);
-		DataOutputStream dos = new DataOutputStream(fdos);
-		for (int i = 0; i < 10; i++) {
-			dos.writeLong(ThreadLocalRandom.current().nextLong());
-		}
-
-		try {
-			SavepointStore.loadSavepoint(filePath.toString(), Thread.currentThread().getContextClassLoader());
-			fail("Did not throw expected Exception");
-		} catch (RuntimeException e) {
-			assertTrue(e.getMessage().contains("Flink 1.0") && e.getMessage().contains("Unexpected magic number"));
-		}
-	}
-
-	/**
-	 * Tests addition of a new savepoint version.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testMultipleSavepointVersions() throws Exception {
-		Field field = SavepointSerializers.class.getDeclaredField("SERIALIZERS");
-		field.setAccessible(true);
-		Map<Integer, SavepointSerializer<?>> serializers = (Map<Integer, SavepointSerializer<?>>) field.get(null);
-
-		assertTrue(serializers.size() >= 1);
-
-		String root = tmp.getRoot().getAbsolutePath();
-		File rootFile = new File(root);
-
-		// New savepoint type for test
-		int version = ThreadLocalRandom.current().nextInt(Integer.MAX_VALUE); // make this a positive number
-		long checkpointId = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); // make this a positive number
-
-		// Add serializer
-		serializers.put(version, NewSavepointSerializer.INSTANCE);
-
-		String savepointDirectory1 = SavepointStore.createSavepointDirectory(root, new JobID());
-		TestSavepoint newSavepoint = new TestSavepoint(version, checkpointId);
-		String pathNewSavepoint = SavepointStore.storeSavepoint(savepointDirectory1, newSavepoint);
-
-		File[] list = rootFile.listFiles();
-
-		assertNotNull(list);
-		assertEquals(1, list.length);
-
-		// Savepoint v0
-		String savepointDirectory2 = SavepointStore.createSavepointDirectory(root, new JobID());
-		SavepointV2 savepoint = new SavepointV2(
-			checkpointId,
-			CheckpointTestUtils.createOperatorStates(4, 32),
-			Collections.<MasterState>emptyList());
-		String pathSavepoint = SavepointStore.storeSavepoint(savepointDirectory2, savepoint);
-
-		list = rootFile.listFiles();
-
-		assertNotNull(list);
-		assertEquals(2, list.length);
-
-		// Load
-		Savepoint loaded = SavepointStore.loadSavepoint(pathNewSavepoint, Thread.currentThread().getContextClassLoader());
-		assertEquals(newSavepoint, loaded);
-
-		loaded = SavepointStore.loadSavepoint(pathSavepoint, Thread.currentThread().getContextClassLoader());
-		assertEquals(savepoint.getCheckpointId(), loaded.getCheckpointId());
-		assertEquals(savepoint.getTaskStates(), loaded.getTaskStates());
-		assertEquals(savepoint.getMasterStates(), loaded.getMasterStates());
-	}
-
-	/**
-	 * Tests that an exception during store cleans up the created savepoint file.
-	 */
-	@Test
-	public void testCleanupOnStoreFailure() throws Exception {
-		Field field = SavepointSerializers.class.getDeclaredField("SERIALIZERS");
-		field.setAccessible(true);
-		Map<Integer, SavepointSerializer<?>> serializers = (Map<Integer, SavepointSerializer<?>>) field.get(null);
-
-		String target = tmp.getRoot().getAbsolutePath();
-
-		final int version = 123123;
-		SavepointSerializer<TestSavepoint> serializer = mock(SavepointSerializer.class);
-		doThrow(new RuntimeException("Test Exception")).when(serializer)
-				.serialize(Matchers.any(TestSavepoint.class), any(DataOutputStream.class));
-
-		serializers.put(version, serializer);
-
-		Savepoint savepoint = new TestSavepoint(version, 12123123);
-
-		assertEquals(0, tmp.getRoot().listFiles().length);
-
-		try {
-			SavepointStore.storeSavepoint(target, savepoint);
-		} catch (Throwable ignored) {
-		}
-
-		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
-		SavepointV2 savepoint = new SavepointV2(
-			1929292,
-			CheckpointTestUtils.createOperatorStates(4, 24),
-			Collections.<MasterState>emptyList());
-
-		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();
-
-		@Override
-		public void serialize(TestSavepoint savepoint, DataOutputStream dos) throws IOException {
-			dos.writeInt(savepoint.version);
-			dos.writeLong(savepoint.checkpointId);
-		}
-
-		@Override
-		public TestSavepoint deserialize(DataInputStream dis, ClassLoader userCL) throws IOException {
-			int version = dis.readInt();
-			long checkpointId = dis.readLong();
-			return new TestSavepoint(version, checkpointId);
-		}
-
-	}
-
-	private static class TestSavepoint implements Savepoint {
-
-		private final int version;
-		private final long checkpointId;
-
-		public TestSavepoint(int version, long checkpointId) {
-			this.version = version;
-			this.checkpointId = checkpointId;
-		}
-
-		@Override
-		public int getVersion() {
-			return version;
-		}
-
-		@Override
-		public long getCheckpointId() {
-			return checkpointId;
-		}
-
-		@Override
-		public Collection<TaskState> getTaskStates() {
-			return Collections.emptyList();
-		}
-
-		@Override
-		public Collection<MasterState> getMasterStates() {
-			return Collections.emptyList();
-		}
-
-		@Override
-		public Collection<OperatorState> getOperatorStates() {
-			return null;
-		}
-
-		@Override
-		public void dispose() {
-		}
-
-		@Override
-		public boolean equals(Object o) {
-			if (this == o) {
-				return true;
-			}
-			if (o == null || getClass() != o.getClass()) {
-				return false;
-			}
-			TestSavepoint that = (TestSavepoint) o;
-			return version == that.version && checkpointId == that.checkpointId;
-
-		}
-
-		@Override
-		public int hashCode() {
-			int result = version;
-			result = 31 * result + (int) (checkpointId ^ (checkpointId >>> 32));
-			return result;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 3fe8613..ba93b7b 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
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.runtime.executiongraph;
 
 import org.apache.flink.api.common.ArchivedExecutionConfig;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.LongCounter;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook;
@@ -42,7 +42,6 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.SerializedValue;
@@ -126,22 +125,16 @@ public class ArchivedExecutionGraphTest extends TestLogger {
 			100,
 			100,
 			1,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			Collections.<ExecutionJobVertex>emptyList(),
 			Collections.<ExecutionJobVertex>emptyList(),
 			Collections.<ExecutionJobVertex>emptyList(),
 			Collections.<MasterTriggerRestoreHook<?>>emptyList(),
 			new StandaloneCheckpointIDCounter(),
 			new StandaloneCompletedCheckpointStore(1),
-			null,
 			new MemoryStateBackend(),
 			statsTracker);
 
-		Map<String, Accumulator<?, ?>> userAccumulators = new HashMap<>();
-		userAccumulators.put("userAcc", new LongCounter(64));
-
-		Execution executionWithAccumulators = runtimeGraph.getJobVertex(v1ID).getTaskVertices()[0].getCurrentExecutionAttempt();
-
 		runtimeGraph.setJsonPlan("{}");
 
 		runtimeGraph.getJobVertex(v2ID).getTaskVertices()[0].getCurrentExecutionAttempt().fail(new RuntimeException("This exception was thrown on purpose."));

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index 12e9b5d..55166d5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobWriter;
 import org.apache.flink.runtime.blob.PermanentBlobService;
 import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
 import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
@@ -50,7 +51,6 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway;
@@ -690,7 +690,7 @@ public class ExecutionGraphDeploymentTest extends TestLogger {
 					10 * 60 * 1000,
 					0,
 					1,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					false),
 				null));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java
index 4725296..a3a26f3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.ManuallyTriggeredDirectExecutor;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.checkpoint.PendingCheckpoint;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter;
@@ -41,7 +42,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
@@ -77,7 +77,7 @@ import static org.mockito.Mockito.when;
 /**
  * These tests make sure that global failover (restart all) always takes precedence over
  * local recovery strategies.
- * 
+ *
  * <p>This test must be in the package it resides in, because it uses package-private methods
  * from the ExecutionGraph classes.
  */
@@ -312,7 +312,7 @@ public class IndividualRestartsConcurrencyTest extends TestLogger {
 			100000L,
 			1L,
 			3,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			true);
 
 		final ExecutionGraph graph = createSampleGraph(
@@ -330,14 +330,13 @@ public class IndividualRestartsConcurrencyTest extends TestLogger {
 			checkpointCoordinatorConfiguration.getCheckpointTimeout(),
 			checkpointCoordinatorConfiguration.getMinPauseBetweenCheckpoints(),
 			checkpointCoordinatorConfiguration.getMaxConcurrentCheckpoints(),
-			checkpointCoordinatorConfiguration.getExternalizedCheckpointSettings(),
+			checkpointCoordinatorConfiguration.getCheckpointRetentionPolicy(),
 			allVertices,
 			allVertices,
 			allVertices,
 			Collections.emptyList(),
 			standaloneCheckpointIDCounter,
 			new StandaloneCompletedCheckpointStore(1),
-			"",
 			new MemoryStateBackend(),
 			new CheckpointStatsTracker(
 				1,

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
index 721b8f1..51e7fec 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.SerializedValue;
@@ -47,7 +48,7 @@ public class JobCheckpointingSettingsTest {
 				1231,
 				112,
 				12,
-				ExternalizedCheckpointSettings.externalizeCheckpoints(true),
+				CheckpointRetentionPolicy.RETAIN_ON_FAILURE,
 				false),
 			new SerializedValue<>(new MemoryStateBackend()));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 79f7342..af9c8b4 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
@@ -34,6 +34,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter;
@@ -56,7 +57,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
@@ -252,7 +252,7 @@ public class JobManagerHARecoveryTest extends TestLogger {
 						10L * 60L * 1000L,
 						0L,
 						1,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 452afd6..f8da2a7 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
@@ -34,6 +34,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions.CheckpointType;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
 import org.apache.flink.runtime.clusterframework.messages.NotifyResourceStarted;
 import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
@@ -61,7 +62,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.jobmanager.JobManagerHARecoveryTest.BlockingStatefulInvokable;
 import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
@@ -893,7 +893,7 @@ public class JobManagerTest extends TestLogger {
 					3600000,
 					0,
 					Integer.MAX_VALUE,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					true),
 					null);
 
@@ -921,7 +921,7 @@ public class JobManagerTest extends TestLogger {
 				if (cancelResp instanceof CancellationFailure) {
 					CancellationFailure failure = (CancellationFailure) cancelResp;
 					if (failure.cause().getMessage().contains(CheckpointDeclineReason.NOT_ALL_REQUIRED_TASKS_RUNNING.message())) {
-						Thread.sleep(200); // wait and retry
+						Thread.sleep(10); // wait and retry
 					} else {
 						failure.cause().printStackTrace();
 						fail("Failed to cancel job: " + failure.cause().getMessage());
@@ -983,7 +983,7 @@ public class JobManagerTest extends TestLogger {
 			3600000,
 			0,
 			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
+			CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 			true);
 
 		JobCheckpointingSettings snapshottingSettings = new JobCheckpointingSettings(
@@ -1105,7 +1105,7 @@ public class JobManagerTest extends TestLogger {
 					3600000,
 					0,
 					Integer.MAX_VALUE,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					true),
 				null);
 
@@ -1220,7 +1220,7 @@ public class JobManagerTest extends TestLogger {
 						360000,
 						0,
 						Integer.MAX_VALUE,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null);
 
@@ -1336,7 +1336,7 @@ public class JobManagerTest extends TestLogger {
 						360000,
 						0,
 						Integer.MAX_VALUE,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null);
 
@@ -1384,7 +1384,7 @@ public class JobManagerTest extends TestLogger {
 						360000,
 						0,
 						Integer.MAX_VALUE,
-						ExternalizedCheckpointSettings.none(),
+						CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 						true),
 					null);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/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 0ca83ae..508ccd0 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
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.blob.PermanentBlobKey;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
@@ -32,7 +33,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -42,6 +42,7 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.NetUtils;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -235,7 +236,7 @@ public class JobSubmitTest {
 					5000,
 					0L,
 					10,
-					ExternalizedCheckpointSettings.none(),
+					CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 					true),
 				null));
 		return jg;

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
index 9e801a3..13a3194 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
@@ -21,11 +21,11 @@ package org.apache.flink.runtime.rest.handler.legacy;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
 import org.apache.flink.util.TestLogger;
 
@@ -97,7 +97,7 @@ public class JobCancellationWithSavepointHandlersTest extends TestLogger {
 				timeout,
 				1L,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				true));
 
 		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
@@ -131,7 +131,7 @@ public class JobCancellationWithSavepointHandlersTest extends TestLogger {
 				timeout,
 				1L,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				true));
 
 		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor, "the-default-directory");
@@ -184,7 +184,7 @@ public class JobCancellationWithSavepointHandlersTest extends TestLogger {
 				1L,
 				1L,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				true));
 
 		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
@@ -316,7 +316,7 @@ public class JobCancellationWithSavepointHandlersTest extends TestLogger {
 				1L,
 				1L,
 				1,
-				ExternalizedCheckpointSettings.none(),
+				CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
 				true));
 
 		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
index c7560da..47ebb18 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
@@ -19,10 +19,10 @@
 package org.apache.flink.runtime.rest.handler.legacy.checkpoints;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
 import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
 import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
@@ -55,7 +55,7 @@ public class CheckpointConfigHandlerTest {
 		AccessExecutionGraph graph = graphAndSettings.graph;
 		when(graph.getJobID()).thenReturn(new JobID());
 		CheckpointCoordinatorConfiguration chkConfig = graphAndSettings.jobCheckpointingConfiguration;
-		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
+		CheckpointRetentionPolicy retentionPolicy = graphAndSettings.retentionPolicy;
 
 		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
 		Assert.assertEquals(1, archives.size());
@@ -73,8 +73,8 @@ public class CheckpointConfigHandlerTest {
 
 		JsonNode externalizedNode = rootNode.get("externalization");
 		Assert.assertNotNull(externalizedNode);
-		Assert.assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
-		Assert.assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
+		Assert.assertEquals(retentionPolicy != CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, externalizedNode.get("enabled").asBoolean());
+		Assert.assertEquals(retentionPolicy != CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION, externalizedNode.get("delete_on_cancellation").asBoolean());
 
 	}
 
@@ -139,7 +139,7 @@ public class CheckpointConfigHandlerTest {
 		GraphAndSettings graphAndSettings = createGraphAndSettings(true, false);
 
 		AccessExecutionGraph graph = graphAndSettings.graph;
-		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
+		CheckpointRetentionPolicy retentionPolicy = graphAndSettings.retentionPolicy;
 
 		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphCache.class), Executors.directExecutor());
 		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
@@ -147,8 +147,8 @@ public class CheckpointConfigHandlerTest {
 		ObjectMapper mapper = new ObjectMapper();
 		JsonNode externalizedNode = mapper.readTree(json).get("externalization");
 		assertNotNull(externalizedNode);
-		assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
-		assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
+		assertEquals(retentionPolicy != CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, externalizedNode.get("enabled").asBoolean());
+		assertEquals(retentionPolicy != CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION, externalizedNode.get("delete_on_cancellation").asBoolean());
 	}
 
 	private static GraphAndSettings createGraphAndSettings(boolean externalized, boolean exactlyOnce) {
@@ -156,36 +156,37 @@ public class CheckpointConfigHandlerTest {
 		long timeout = 996979L;
 		long minPause = 119191919L;
 		int maxConcurrent = 12929329;
-		ExternalizedCheckpointSettings externalizedSetting = externalized
-			? ExternalizedCheckpointSettings.externalizeCheckpoints(true)
-			: ExternalizedCheckpointSettings.none();
+
+		CheckpointRetentionPolicy retentionPolicy = externalized
+			? CheckpointRetentionPolicy.RETAIN_ON_FAILURE
+			: CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION;
 
 		CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration(
 			interval,
 			timeout,
 			minPause,
 			maxConcurrent,
-			externalizedSetting,
+			retentionPolicy,
 			exactlyOnce);
 
 		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
 		when(graph.getCheckpointCoordinatorConfiguration()).thenReturn(chkConfig);
 
-		return new GraphAndSettings(graph, chkConfig, externalizedSetting);
+		return new GraphAndSettings(graph, chkConfig, retentionPolicy);
 	}
 
 	private static class GraphAndSettings {
 		public final AccessExecutionGraph graph;
 		public final CheckpointCoordinatorConfiguration jobCheckpointingConfiguration;
-		public final ExternalizedCheckpointSettings externalizedSettings;
+		public final CheckpointRetentionPolicy retentionPolicy;
 
 		public GraphAndSettings(
 				AccessExecutionGraph graph,
 				CheckpointCoordinatorConfiguration jobCheckpointingConfiguration,
-				ExternalizedCheckpointSettings externalizedSettings) {
+				CheckpointRetentionPolicy retentionPolicy) {
 			this.graph = graph;
 			this.jobCheckpointingConfiguration = jobCheckpointingConfiguration;
-			this.externalizedSettings = externalizedSettings;
+			this.retentionPolicy = retentionPolicy;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
index 7cc8efb..ef65e43 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.rest.handler.legacy.checkpoints;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
 import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
@@ -167,7 +168,8 @@ public class CheckpointStatsDetailsHandlerTest {
 		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
 		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
 		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(checkpoint.getProperties()).thenReturn(
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		when(checkpoint.getTriggerTimestamp()).thenReturn(1919191900L);
 		when(checkpoint.getLatestAckTimestamp()).thenReturn(1977791901L);
 		when(checkpoint.getStateSize()).thenReturn(111939272822L);
@@ -234,7 +236,7 @@ public class CheckpointStatsDetailsHandlerTest {
 		CompletedCheckpointStats checkpoint = mock(CompletedCheckpointStats.class);
 		when(checkpoint.getCheckpointId()).thenReturn(1818213L);
 		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forSavepoint());
 		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
 		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
 		when(checkpoint.getStateSize()).thenReturn(925281L);
@@ -275,7 +277,7 @@ public class CheckpointStatsDetailsHandlerTest {
 		FailedCheckpointStats checkpoint = mock(FailedCheckpointStats.class);
 		when(checkpoint.getCheckpointId()).thenReturn(1818214L);
 		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forSavepoint());
 		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
 		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
 		when(checkpoint.getStateSize()).thenReturn(925281L);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
index cc90d7f..1d5768b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.rest.handler.legacy.checkpoints;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
 import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
@@ -173,7 +174,7 @@ public class CheckpointStatsHandlerTest {
 		RestoredCheckpointStats latestRestored = mock(RestoredCheckpointStats.class);
 		when(latestRestored.getCheckpointId()).thenReturn(1199L);
 		when(latestRestored.getRestoreTimestamp()).thenReturn(434242L);
-		when(latestRestored.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(latestRestored.getProperties()).thenReturn(CheckpointProperties.forSavepoint());
 		when(latestRestored.getExternalPath()).thenReturn("restored savepoint path");
 
 		// History
@@ -183,7 +184,8 @@ public class CheckpointStatsHandlerTest {
 		PendingCheckpointStats inProgress = mock(PendingCheckpointStats.class);
 		when(inProgress.getCheckpointId()).thenReturn(1992141L);
 		when(inProgress.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(inProgress.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(inProgress.getProperties()).thenReturn(
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		when(inProgress.getTriggerTimestamp()).thenReturn(1919191900L);
 		when(inProgress.getLatestAckTimestamp()).thenReturn(1977791901L);
 		when(inProgress.getStateSize()).thenReturn(111939272822L);
@@ -195,7 +197,7 @@ public class CheckpointStatsHandlerTest {
 		CompletedCheckpointStats completedSavepoint = mock(CompletedCheckpointStats.class);
 		when(completedSavepoint.getCheckpointId()).thenReturn(1322139L);
 		when(completedSavepoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
-		when(completedSavepoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(completedSavepoint.getProperties()).thenReturn(CheckpointProperties.forSavepoint());
 		when(completedSavepoint.getTriggerTimestamp()).thenReturn(191900L);
 		when(completedSavepoint.getLatestAckTimestamp()).thenReturn(197791901L);
 		when(completedSavepoint.getStateSize()).thenReturn(1119822L);
@@ -209,7 +211,8 @@ public class CheckpointStatsHandlerTest {
 		FailedCheckpointStats failed = mock(FailedCheckpointStats.class);
 		when(failed.getCheckpointId()).thenReturn(110719L);
 		when(failed.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
-		when(failed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(failed.getProperties()).thenReturn(
+				CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION));
 		when(failed.getTriggerTimestamp()).thenReturn(191900L);
 		when(failed.getLatestAckTimestamp()).thenReturn(197791901L);
 		when(failed.getStateSize()).thenReturn(1119822L);

http://git-wip-us.apache.org/repos/asf/flink/blob/edc6f100/flink-runtime/src/test/java/org/apache/flink/runtime/state/EmptyStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/EmptyStreamStateHandle.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/EmptyStreamStateHandle.java
new file mode 100644
index 0000000..4b42959
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/EmptyStreamStateHandle.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.core.fs.FSDataInputStream;
+
+import java.io.IOException;
+
+/**
+ * A simple dummy implementation of a stream state handle that can be passed in tests.
+ * The handle cannot open an input stream.
+ */
+public class EmptyStreamStateHandle implements StreamStateHandle {
+
+	private static final long serialVersionUID = 0L;
+
+	@Override
+	public FSDataInputStream openInputStream() throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public void discardState() {}
+
+	@Override
+	public long getStateSize() {
+		return 0;
+	}
+}


[10/17] flink git commit: [FLINK-5823] [checkpoints] Make RocksDB state backend configurable

Posted by se...@apache.org.
[FLINK-5823] [checkpoints] Make RocksDB state backend configurable


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

Branch: refs/heads/master
Commit: d19525e90e69ddd257d47371b8ea0319fa4673c8
Parents: 1931993
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Oct 26 15:43:23 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 18 18:08:03 2018 +0100

----------------------------------------------------------------------
 .../streaming/state/RocksDBStateBackend.java    | 159 +++++++++++++++----
 .../state/RocksDBStateBackendFactory.java       |  31 +---
 .../state/RocksDBAsyncSnapshotTest.java         |   7 +
 .../state/RocksDBStateBackendConfigTest.java    |  61 +++++--
 .../state/RocksDBStateBackendFactoryTest.java   | 139 +++++++++++++++-
 .../state/RocksDBStateBackendTest.java          |   4 +-
 6 files changed, 327 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/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 a6552bc..6bcd595 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
@@ -20,15 +20,20 @@ package org.apache.flink.contrib.streaming.state;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.util.AbstractID;
 
@@ -67,7 +72,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * using the methods {@link #setPredefinedOptions(PredefinedOptions)} and
  * {@link #setOptions(OptionsFactory)}.
  */
-public class RocksDBStateBackend extends AbstractStateBackend {
+public class RocksDBStateBackend extends AbstractStateBackend implements ConfigurableStateBackend {
 
 	private static final long serialVersionUID = 1L;
 
@@ -83,9 +88,11 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	// -- configuration values, set in the application / configuration
 
 	/** The state backend that we use for creating checkpoint streams. */
-	private final AbstractStateBackend checkpointStreamBackend;
+	private final StateBackend checkpointStreamBackend;
 
-	/** Base paths for RocksDB directory, as configured. May be null. */
+	/** Base paths for RocksDB directory, as configured.
+	 * Null if not yet set, in which case the configuration values will be used.
+	 * The configuration defaults to the TaskManager's temp directories. */
 	@Nullable
 	private Path[] localRocksDbDirectories;
 
@@ -96,8 +103,10 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	@Nullable
 	private OptionsFactory optionsFactory;
 
-	/** True if incremental checkpointing is enabled. */
-	private boolean enableIncrementalCheckpointing;
+	/** True if incremental checkpointing is enabled.
+	 * Null if not yet set, in which case the configuration values will be used. */
+	@Nullable
+	private Boolean enableIncrementalCheckpointing;
 
 	// -- runtime values, set on TaskManager when initializing / using the backend
 
@@ -107,8 +116,9 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	/** JobID for uniquifying backup paths. */
 	private transient JobID jobId;
 
+	/** The index of the next directory to be used from {@link #initializedDbBasePaths}.*/
 	private transient int nextDirectory;
-	
+
 	/** Whether we already lazily initialized our local storage directories. */
 	private transient boolean isInitialized;
 
@@ -127,7 +137,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
 	public RocksDBStateBackend(String checkpointDataUri) throws IOException {
-		this(new Path(checkpointDataUri).toUri(), false);
+		this(new Path(checkpointDataUri).toUri());
 	}
 
 	/**
@@ -160,7 +170,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
 	 */
 	public RocksDBStateBackend(URI checkpointDataUri) throws IOException {
-		this(new FsStateBackend(checkpointDataUri), false);
+		this(new FsStateBackend(checkpointDataUri));
 	}
 
 	/**
@@ -188,7 +198,7 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * <p>The snapshots of the RocksDB state will be stored using the given backend's
 	 * {@link AbstractStateBackend#createStreamFactory(JobID, String) checkpoint stream}.
 	 *
-	 * @param checkpointStreamBackend The backend to store the
+	 * @param checkpointStreamBackend The backend write the checkpoint streams to.
 	 */
 	public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend) {
 		this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend);
@@ -202,18 +212,92 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 	 * <p>The snapshots of the RocksDB state will be stored using the given backend's
 	 * {@link AbstractStateBackend#createStreamFactory(JobID, String) checkpoint stream}.
 	 *
-	 * @param checkpointStreamBackend The backend to store the
-	 * @param enableIncrementalCheckpointing True if incremental checkponting is enabled
+	 * @param checkpointStreamBackend The backend write the checkpoint streams to.
+	 * @param enableIncrementalCheckpointing True if incremental checkpointing is enabled.
 	 */
 	public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend, boolean enableIncrementalCheckpointing) {
 		this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend);
 		this.enableIncrementalCheckpointing = enableIncrementalCheckpointing;
 	}
 
+	/**
+	 * Private constructor that creates a re-configured copy of the state backend.
+	 *
+	 * @param original The state backend to re-configure.
+	 * @param config The configuration.
+	 */
+	private RocksDBStateBackend(RocksDBStateBackend original, Configuration config) {
+		// reconfigure the state backend backing the streams
+		final StateBackend originalStreamBackend = original.checkpointStreamBackend;
+		this.checkpointStreamBackend = originalStreamBackend instanceof ConfigurableStateBackend ?
+				((ConfigurableStateBackend) originalStreamBackend).configure(config) :
+				originalStreamBackend;
+
+		// configure incremental checkpoints
+		if (original.enableIncrementalCheckpointing != null) {
+			this.enableIncrementalCheckpointing = original.enableIncrementalCheckpointing;
+		}
+		else {
+			this.enableIncrementalCheckpointing =
+					config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS);
+		}
+
+		// configure local directories
+		if (original.localRocksDbDirectories != null) {
+			this.localRocksDbDirectories = original.localRocksDbDirectories;
+		}
+		else {
+			final String rocksdbLocalPaths = config.getString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES);
+			if (rocksdbLocalPaths != null) {
+				String[] directories = rocksdbLocalPaths.split(",|" + File.pathSeparator);
+
+				try {
+					setDbStoragePaths(directories);
+				}
+				catch (IllegalArgumentException e) {
+					throw new IllegalConfigurationException("Invalid configuration for RocksDB state " +
+							"backend's local storage directories: " + e.getMessage(), e);
+				}
+			}
+		}
+
+		// copy remaining settings
+		this.predefinedOptions = original.predefinedOptions;
+		this.optionsFactory = original.optionsFactory;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reconfiguration
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a copy of this state backend that uses the values defined in the configuration
+	 * for fields where that were not yet specified in this state backend.
+	 *
+	 * @param config the configuration
+	 * @return The re-configured variant of the state backend
+	 */
+	@Override
+	public RocksDBStateBackend configure(Configuration config) {
+		return new RocksDBStateBackend(this, config);
+	}
+
 	// ------------------------------------------------------------------------
 	//  State backend methods
 	// ------------------------------------------------------------------------
 
+	/**
+	 * Gets the state backend that this RocksDB state backend uses to persist
+	 * its bytes to.
+	 *
+	 * <p>This RocksDB state backend only implements the RocksDB specific parts, it
+	 * relies on the 'CheckpointBackend' to persist the checkpoint and savepoint bytes
+	 * streams.
+	 */
+	public StateBackend getCheckpointBackend() {
+		return checkpointStreamBackend;
+	}
+
 	private void lazyInitializeForJob(
 			Environment env,
 			String operatorIdentifier) throws IOException {
@@ -314,7 +398,20 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 				numberOfKeyGroups,
 				keyGroupRange,
 				env.getExecutionConfig(),
-				enableIncrementalCheckpointing);
+				isIncrementalCheckpointsEnabled());
+	}
+
+	@Override
+	public OperatorStateBackend createOperatorStateBackend(
+			Environment env,
+			String operatorIdentifier) throws Exception {
+
+		//the default for RocksDB; eventually there can be a operator state backend based on RocksDB, too.
+		final boolean asyncSnapshots = true;
+		return new DefaultOperatorStateBackend(
+				env.getUserClassLoader(),
+				env.getExecutionConfig(),
+				asyncSnapshots);
 	}
 
 	// ------------------------------------------------------------------------
@@ -390,6 +487,18 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 		}
 	}
 
+	/**
+	 * Gets whether incremental checkpoints are enabled for this state backend.
+	 */
+	public boolean isIncrementalCheckpointsEnabled() {
+		if (enableIncrementalCheckpointing != null) {
+			return enableIncrementalCheckpointing;
+		}
+		else {
+			return CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue();
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  Parametrize with RocksDB Options
 	// ------------------------------------------------------------------------
@@ -480,27 +589,17 @@ public class RocksDBStateBackend extends AbstractStateBackend {
 		return opt;
 	}
 
-	@Override
-	public OperatorStateBackend createOperatorStateBackend(
-		Environment env,
-		String operatorIdentifier) throws Exception {
-
-		//the default for RocksDB; eventually there can be a operator state backend based on RocksDB, too.
-		final boolean asyncSnapshots = true;
-		return new DefaultOperatorStateBackend(
-			env.getUserClassLoader(),
-			env.getExecutionConfig(),
-			asyncSnapshots);
-	}
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
 
 	@Override
 	public String toString() {
-		return "RocksDB State Backend {" +
-			"isInitialized=" + isInitialized +
-			", configuredDbBasePaths=" + Arrays.toString(localRocksDbDirectories) +
-			", initializedDbBasePaths=" + Arrays.toString(initializedDbBasePaths) +
-			", checkpointStreamBackend=" + checkpointStreamBackend +
-			'}';
+		return "RocksDBStateBackend{" +
+				"checkpointStreamBackend=" + checkpointStreamBackend +
+				", localRocksDbDirectories=" + Arrays.toString(localRocksDbDirectories) +
+				", enableIncrementalCheckpointing=" + enableIncrementalCheckpointing +
+				'}';
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/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 de5be9a..94e15fa 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
@@ -21,13 +21,8 @@ package org.apache.flink.contrib.streaming.state;
 import org.apache.flink.configuration.CheckpointingOptions;
 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.StateBackendFactory;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
 import java.io.IOException;
 
 /**
@@ -36,39 +31,19 @@ import java.io.IOException;
  */
 public class RocksDBStateBackendFactory implements StateBackendFactory<RocksDBStateBackend> {
 
-	protected static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackendFactory.class);
-
 	@Override
 	public RocksDBStateBackend createFromConfig(Configuration config)
 			throws IllegalConfigurationException, IOException {
 
+		// we need to explicitly read the checkpoint directory here, because that
+		// is a required constructor parameter
 		final String checkpointDirURI = config.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY);
-		final String rocksdbLocalPaths = config.getString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES);
-		final boolean incrementalCheckpoints = config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS);
-
 		if (checkpointDirURI == null) {
 			throw new IllegalConfigurationException(
 				"Cannot create the RocksDB state backend: The configuration does not specify the " +
 				"checkpoint directory '" + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key() + '\'');
 		}
 
-		try {
-			final Path path = new Path(checkpointDirURI);
-			final RocksDBStateBackend backend = new RocksDBStateBackend(path.toUri(), incrementalCheckpoints);
-
-			if (rocksdbLocalPaths != null) {
-				String[] directories = rocksdbLocalPaths.split(",|" + File.pathSeparator);
-				backend.setDbStoragePaths(directories);
-			}
-
-			LOG.info("State backend is set to RocksDB (configured DB storage paths {}, checkpoints to filesystem {} ) ",
-					backend.getDbStoragePaths(), path);
-
-			return backend;
-		}
-		catch (IllegalArgumentException e) {
-			throw new IllegalConfigurationException(
-					"Cannot initialize RocksDB State Backend with URI '" + checkpointDirURI + '.', e);
-		}
+		return new RocksDBStateBackend(checkpointDirURI).configure(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
index 8dc504a..461c526 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.common.typeutils.base.VoidSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
@@ -416,6 +417,12 @@ public class RocksDBAsyncSnapshotTest extends TestLogger {
 		public CheckpointStreamFactory createStreamFactory(JobID jobId, String operatorIdentifier) throws IOException {
 			return blockerCheckpointStreamFactory;
 		}
+
+		@Override
+		public BlockingStreamMemoryStateBackend configure(Configuration config) {
+			// retain this instance, no re-configuration!
+			return this;
+		}
 	}
 
 	private static class AsyncCheckpointOperator

http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
index 853d80f..eace5ff 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.contrib.streaming.state;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.Environment;
@@ -30,11 +31,11 @@ import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
+import org.apache.flink.util.IOUtils;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -49,6 +50,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.startsWith;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -60,14 +62,12 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-
 /**
  * Tests for configuring the RocksDB State Backend.
  */
 @SuppressWarnings("serial")
 public class RocksDBStateBackendConfigTest {
 
-
 	@Rule
 	public TemporaryFolder tempFolder = new TemporaryFolder();
 
@@ -76,6 +76,14 @@ public class RocksDBStateBackendConfigTest {
 	// ------------------------------------------------------------------------
 
 	@Test
+	public void testDefaultsInSync() throws Exception {
+		final boolean defaultIncremental = CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue();
+
+		RocksDBStateBackend backend = new RocksDBStateBackend(tempFolder.newFolder().toURI());
+		assertEquals(defaultIncremental, backend.isIncrementalCheckpointsEnabled());
+	}
+
+	@Test
 	public void testSetDbPath() throws Exception {
 		String checkpointPath = tempFolder.newFolder().toURI().toString();
 		File testDir1 = tempFolder.newFolder();
@@ -139,8 +147,6 @@ public class RocksDBStateBackendConfigTest {
 	/**
 	 * This tests whether the RocksDB backends uses the temp directories that are provided
 	 * from the {@link Environment} when no db storage path is set.
-	 *
-	 * @throws Exception
 	 */
 	@Test
 	public void testUseTempDirectories() throws Exception {
@@ -214,7 +220,6 @@ public class RocksDBStateBackendConfigTest {
 		finally {
 			//noinspection ResultOfMethodCallIgnored
 			targetDir.setWritable(true, false);
-			FileUtils.deleteDirectory(targetDir);
 		}
 	}
 
@@ -256,8 +261,6 @@ public class RocksDBStateBackendConfigTest {
 		} finally {
 			//noinspection ResultOfMethodCallIgnored
 			targetDir1.setWritable(true, false);
-			FileUtils.deleteDirectory(targetDir1);
-			FileUtils.deleteDirectory(targetDir2);
 		}
 	}
 
@@ -329,16 +332,48 @@ public class RocksDBStateBackendConfigTest {
 	@Test
 	public void testPredefinedOptionsEnum() {
 		for (PredefinedOptions o : PredefinedOptions.values()) {
-			DBOptions opt = o.createDBOptions();
-			try {
+			try (DBOptions opt = o.createDBOptions()) {
 				assertNotNull(opt);
-			} finally {
-				opt.dispose();
 			}
 		}
 	}
 
 	// ------------------------------------------------------------------------
+	//  Reconfiguration
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testRocksDbReconfigurationCopiesExistingValues() throws Exception {
+		final FsStateBackend checkpointBackend = new FsStateBackend(tempFolder.newFolder().toURI().toString());
+		final boolean incremental = !CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue();
+
+		final RocksDBStateBackend original = new RocksDBStateBackend(checkpointBackend, incremental);
+
+		// these must not be the default options
+		final PredefinedOptions predOptions = PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM;
+		assertNotEquals(predOptions, original.getPredefinedOptions());
+		original.setPredefinedOptions(predOptions);
+
+		final OptionsFactory optionsFactory = mock(OptionsFactory.class);
+		original.setOptions(optionsFactory);
+
+		final String[] localDirs = new String[] {
+				tempFolder.newFolder().getAbsolutePath(), tempFolder.newFolder().getAbsolutePath() };
+		original.setDbStoragePaths(localDirs);
+
+		RocksDBStateBackend copy = original.configure(new Configuration());
+
+		assertEquals(original.isIncrementalCheckpointsEnabled(), copy.isIncrementalCheckpointsEnabled());
+		assertArrayEquals(original.getDbStoragePaths(), copy.getDbStoragePaths());
+		assertEquals(original.getOptions(), copy.getOptions());
+		assertEquals(original.getPredefinedOptions(), copy.getPredefinedOptions());
+
+		FsStateBackend copyCheckpointBackend = (FsStateBackend) copy.getCheckpointBackend();
+		assertEquals(checkpointBackend.getCheckpointPath(), copyCheckpointBackend.getCheckpointPath());
+		assertEquals(checkpointBackend.getSavepointPath(), copyCheckpointBackend.getSavepointPath());
+	}
+
+	// ------------------------------------------------------------------------
 	//  Contained Non-partitioned State Backend
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
index 5a937c4..7612c4c 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
@@ -18,22 +18,159 @@
 
 package org.apache.flink.contrib.streaming.state;
 
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
+import org.apache.flink.runtime.state.filesystem.AbstractFileStateBackend;
+
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for the RocksDBStateBackendFactory.
  */
 public class RocksDBStateBackendFactoryTest {
 
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	private final ClassLoader cl = getClass().getClassLoader();
+
+	private final String backendKey = CheckpointingOptions.STATE_BACKEND.key();
+
+	// ------------------------------------------------------------------------
+
 	@Test
 	public void testFactoryName() {
 		// construct the name such that it will not be automatically adjusted on refactorings
 		String factoryName = "org.apache.flink.contrib.streaming.state.Roc";
 		factoryName += "ksDBStateBackendFactory";
 
-		// !!! if this fails, the code in StreamTask.createStateBackend() must be adjusted
+		// !!! if this fails, the code in StateBackendLoader must be adjusted
 		assertEquals(factoryName, RocksDBStateBackendFactory.class.getName());
 	}
+
+	/**
+	 * Validates loading a file system state backend with additional parameters from the cluster configuration.
+	 */
+	@Test
+	public void testLoadFileSystemStateBackend() throws Exception {
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String localDir1 = tmp.newFolder().getAbsolutePath();
+		final String localDir2 = tmp.newFolder().getAbsolutePath();
+		final String localDirs = localDir1 + File.pathSeparator + localDir2;
+		final boolean incremental = !CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue();
+
+		final Path expectedCheckpointsPath = new Path(checkpointDir);
+		final Path expectedSavepointsPath = new Path(savepointDir);
+
+		// 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, "rocksdb");
+		config1.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config1.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config1.setString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES, localDirs);
+		config1.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, incremental);
+
+		final Configuration config2 = new Configuration();
+		config2.setString(backendKey, RocksDBStateBackendFactory.class.getName());
+		config2.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
+		config2.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config2.setString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES, localDirs);
+		config2.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, incremental);
+
+		StateBackend backend1 = StateBackendLoader.loadStateBackendFromConfig(config1, cl, null);
+		StateBackend backend2 = StateBackendLoader.loadStateBackendFromConfig(config2, cl, null);
+
+		assertTrue(backend1 instanceof RocksDBStateBackend);
+		assertTrue(backend2 instanceof RocksDBStateBackend);
+
+		RocksDBStateBackend fs1 = (RocksDBStateBackend) backend1;
+		RocksDBStateBackend fs2 = (RocksDBStateBackend) backend2;
+
+		AbstractFileStateBackend fs1back = (AbstractFileStateBackend) fs1.getCheckpointBackend();
+		AbstractFileStateBackend fs2back = (AbstractFileStateBackend) fs2.getCheckpointBackend();
+
+		assertEquals(expectedCheckpointsPath, fs1back.getCheckpointPath());
+		assertEquals(expectedCheckpointsPath, fs2back.getCheckpointPath());
+		assertEquals(expectedSavepointsPath, fs1back.getSavepointPath());
+		assertEquals(expectedSavepointsPath, fs2back.getSavepointPath());
+		assertEquals(incremental, fs1.isIncrementalCheckpointsEnabled());
+		assertEquals(incremental, fs2.isIncrementalCheckpointsEnabled());
+		checkPaths(fs1.getDbStoragePaths(), localDir1, localDir2);
+		checkPaths(fs2.getDbStoragePaths(), localDir1, localDir2);
+	}
+
+	/**
+	 * Validates taking the application-defined file system state backend and adding with additional
+	 * parameters from the cluster configuration, but giving precedence to application-defined
+	 * parameters over configuration-defined parameters.
+	 */
+	@Test
+	public void testLoadFileSystemStateBackendMixed() throws Exception {
+		final String appCheckpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String checkpointDir = new Path(tmp.newFolder().toURI()).toString();
+		final String savepointDir = new Path(tmp.newFolder().toURI()).toString();
+
+		final String localDir1 = tmp.newFolder().getAbsolutePath();
+		final String localDir2 = tmp.newFolder().getAbsolutePath();
+		final String localDir3 = tmp.newFolder().getAbsolutePath();
+		final String localDir4 = tmp.newFolder().getAbsolutePath();
+
+		final boolean incremental = !CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue();
+
+		final Path expectedCheckpointsPath = new Path(appCheckpointDir);
+		final Path expectedSavepointsPath = new Path(savepointDir);
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(appCheckpointDir, incremental);
+		backend.setDbStoragePaths(localDir1, localDir2);
+
+		final Configuration config = new Configuration();
+		config.setString(backendKey, "jobmanager"); // this should not be picked up
+		config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); // this should not be picked up
+		config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir);
+		config.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, !incremental);  // this should not be picked up
+		config.setString(CheckpointingOptions.ROCKSDB_LOCAL_DIRECTORIES, localDir3 + ":" + localDir4);  // this should not be picked up
+
+		final StateBackend loadedBackend =
+				StateBackendLoader.fromApplicationOrConfigOrDefault(backend, config, cl, null);
+		assertTrue(loadedBackend instanceof RocksDBStateBackend);
+
+		final RocksDBStateBackend loadedRocks = (RocksDBStateBackend) loadedBackend;
+
+		assertEquals(incremental, loadedRocks.isIncrementalCheckpointsEnabled());
+		checkPaths(loadedRocks.getDbStoragePaths(), localDir1, localDir2);
+
+		AbstractFileStateBackend fsBackend = (AbstractFileStateBackend) loadedRocks.getCheckpointBackend();
+		assertEquals(expectedCheckpointsPath, fsBackend.getCheckpointPath());
+		assertEquals(expectedSavepointsPath, fsBackend.getSavepointPath());
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static void checkPaths(String[] pathsArray, String... paths) {
+		assertNotNull(pathsArray);
+		assertNotNull(paths);
+
+		assertEquals(pathsArray.length, paths.length);
+
+		HashSet<String> pathsSet = new HashSet<>(Arrays.asList(pathsArray));
+
+		for (String path : paths) {
+			assertTrue(pathsSet.contains(path));
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d19525e9/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
index e1be744..39724c6 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
@@ -110,10 +110,10 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	public boolean enableIncrementalCheckpointing;
 
 	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
 
 	// Store it because we need it for the cleanup test.
-	String dbPath;
+	private String dbPath;
 
 	@Override
 	protected RocksDBStateBackend getStateBackend() throws IOException {