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 > 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 {