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/09/19 07:30:12 UTC
[flink] 02/03: [FLINK-9061] [state] Creation of state payload files
passes optional entropy option to File System
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git
commit cf09494cbda127bbb88e75ed25d4d342b47b8fc7
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Aug 23 20:43:33 2018 +0200
[FLINK-9061] [state] Creation of state payload files passes optional entropy option to File System
---
.../state/filesystem/FsCheckpointStorage.java | 17 +++-
.../filesystem/FsCheckpointStreamFactory.java | 7 +-
.../FsCheckpointStateOutputStreamTest.java | 12 ++-
.../filesystem/FsStateBackendEntropyTest.java | 99 ++++++++++++++++++++++
4 files changed, 126 insertions(+), 9 deletions(-)
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
index af80af7..1549e01 100644
--- 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
@@ -32,6 +32,7 @@ import javax.annotation.Nullable;
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.
@@ -54,11 +55,25 @@ public class FsCheckpointStorage extends AbstractFsCheckpointStorage {
JobID jobId,
int fileSizeThreshold) throws IOException {
+ this(checkpointBaseDirectory.getFileSystem(),
+ checkpointBaseDirectory,
+ defaultSavepointDirectory,
+ jobId,
+ fileSizeThreshold);
+ }
+
+ public FsCheckpointStorage(
+ FileSystem fs,
+ Path checkpointBaseDirectory,
+ @Nullable Path defaultSavepointDirectory,
+ JobID jobId,
+ int fileSizeThreshold) throws IOException {
+
super(jobId, defaultSavepointDirectory);
checkArgument(fileSizeThreshold >= 0);
- this.fileSystem = checkpointBaseDirectory.getFileSystem();
+ this.fileSystem = checkNotNull(fs);
this.checkpointsDirectory = getCheckpointDirectoryForJob(checkpointBaseDirectory, jobId);
this.sharedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_SHARED_STATE_DIR);
this.taskOwnedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_TASK_OWNED_STATE_DIR);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
index 609ef69..228c5b4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
@@ -21,6 +21,7 @@ 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.Path;
+import org.apache.flink.core.fs.WriteOptions;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.CheckpointedStateScope;
import org.apache.flink.runtime.state.StreamStateHandle;
@@ -345,7 +346,11 @@ public class FsCheckpointStreamFactory implements CheckpointStreamFactory {
for (int attempt = 0; attempt < 10; attempt++) {
try {
Path statePath = createStatePath();
- FSDataOutputStream outStream = fs.create(statePath, FileSystem.WriteMode.NO_OVERWRITE);
+ FSDataOutputStream outStream = fs.create(
+ statePath,
+ new WriteOptions()
+ .setOverwrite(FileSystem.WriteMode.NO_OVERWRITE)
+ .setInjectEntropy(true));
// success, managed to open the stream
this.statePath = statePath;
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
index 8bafdf7..c962146 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigConstants;
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.core.fs.WriteOptions;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
import org.apache.flink.runtime.state.StreamStateHandle;
@@ -138,13 +139,13 @@ public class FsCheckpointStateOutputStreamTest {
*/
@Test
public void testCleanupWhenClosingStream() throws IOException {
-
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(FileSystem.WriteMode.class))).thenReturn(outputStream);
+ when(fs.create(pathCaptor.capture(), any(WriteOptions.class))).thenReturn(outputStream);
CheckpointStreamFactory.CheckpointStateOutputStream stream = new FsCheckpointStreamFactory.FsCheckpointStateOutputStream(
Path.fromLocalFile(tempDir.newFolder()),
@@ -154,9 +155,6 @@ public class FsCheckpointStateOutputStreamTest {
// this should create the underlying file stream
stream.write(new byte[] {1, 2, 3, 4, 5});
-
- verify(fs).create(any(Path.class), any(FileSystem.WriteMode.class));
-
stream.close();
verify(fs).delete(eq(pathCaptor.getValue()), anyBoolean());
@@ -170,9 +168,11 @@ public class FsCheckpointStateOutputStreamTest {
final FileSystem fs = mock(FileSystem.class);
final FSDataOutputStream outputStream = mock(FSDataOutputStream.class);
- final ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
+ final ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
when(fs.create(pathCaptor.capture(), any(FileSystem.WriteMode.class))).thenReturn(outputStream);
+ when(fs.create(pathCaptor.capture(), any(WriteOptions.class))).thenReturn(outputStream);
+
doThrow(new IOException("Test IOException.")).when(outputStream).close();
CheckpointStreamFactory.CheckpointStateOutputStream stream = new FsCheckpointStreamFactory.FsCheckpointStateOutputStream(
@@ -184,8 +184,6 @@ public class FsCheckpointStateOutputStreamTest {
// this should create the underlying file stream
stream.write(new byte[] {1, 2, 3, 4, 5});
- verify(fs).create(any(Path.class), any(FileSystem.WriteMode.class));
-
try {
stream.closeAndGetHandle();
fail("Expected IOException");
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
new file mode 100644
index 0000000..d1aa118
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.WriteOptions;
+import org.apache.flink.core.fs.local.LocalFileSystem;
+import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.CheckpointedStateScope;
+
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests verifying that the FsStateBackend passes the entropy injection option
+ * to the FileSystem for state payload files, but not for metadata files.
+ */
+public class FsStateBackendEntropyTest {
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @After
+ public void resetFileSystems() throws Exception {
+ FileSystem.initialize(new Configuration());
+ }
+
+ @Test
+ public void testInjection() throws Exception {
+ FileSystem fs = spy(LocalFileSystem.getSharedInstance());
+ ArgumentCaptor<WriteOptions> optionsCaptor = ArgumentCaptor.forClass(WriteOptions.class);
+
+ Path checkpointDir = Path.fromLocalFile(tmp.newFolder());
+
+ FsCheckpointStorage storage = new FsCheckpointStorage(
+ fs, checkpointDir, null, new JobID(), 1024);
+
+ CheckpointStorageLocation location = storage.initializeLocationForCheckpoint(96562);
+
+ // check entropy in task-owned state
+ try (CheckpointStateOutputStream stream = storage.createTaskOwnedStateStream()) {
+ stream.flush();
+ }
+
+ verify(fs, times(1)).create(any(Path.class), optionsCaptor.capture());
+ assertTrue(optionsCaptor.getValue().isInjectEntropy());
+ reset(fs);
+
+ // check entropy in the exclusive/shared state
+ try (CheckpointStateOutputStream stream =
+ location.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE)) {
+
+ stream.flush();
+ }
+
+ verify(fs, times(1)).create(any(Path.class), optionsCaptor.capture());
+ assertTrue(optionsCaptor.getValue().isInjectEntropy());
+ reset(fs);
+
+ // check that there is no entropy in the metadata
+ // check entropy in the exclusive/shared state
+ try (CheckpointMetadataOutputStream stream = location.createMetadataOutputStream()) {
+ stream.flush();
+ }
+
+ verify(fs, times(0)).create(any(Path.class), any(WriteOptions.class));
+ }
+}