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));
+	}
+}