1996fanrui commented on a change in pull request #13885:
URL: https://github.com/apache/flink/pull/13885#discussion_r516392512



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
##########
@@ -524,13 +524,17 @@ public CheckpointStorageAccess 
createCheckpointStorage(JobID jobId) throws IOExc
                HeapPriorityQueueSetFactory priorityQueueSetFactory =
                        new HeapPriorityQueueSetFactory(keyGroupRange, 
numberOfKeyGroups, 128);
 
+               ReadableConfig config = 
env.getTaskManagerInfo().getConfiguration();
+               int fsReadBufferSize = 
config.get(CheckpointingOptions.FS_READ_BUFFER_SIZE);

Review comment:
       Hi, @sjwiesman , thanks for your comments. I will do it.

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
##########
@@ -157,4 +157,14 @@
                .withDescription(String.format("The default size of the write 
buffer for the checkpoint streams that write to file systems. " +
                        "The actual write buffer size is determined to be the 
maximum of the value of this option and option '%s'.", 
FS_SMALL_FILE_THRESHOLD.key()));
 
+       /**
+        * The default size of the read buffer for the checkpoint streams that 
read from file systems.
+        */
+       @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+       public static final ConfigOption<Integer> FS_READ_BUFFER_SIZE = 
ConfigOptions
+               .key("state.backend.fs.read-buffer-size")
+               .intType()

Review comment:
       This configuration only means reading a buffer size of the file system. 
Should the `memory configuration` contain some `memory size` related config?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
##########
@@ -524,13 +524,17 @@ public CheckpointStorageAccess 
createCheckpointStorage(JobID jobId) throws IOExc
                HeapPriorityQueueSetFactory priorityQueueSetFactory =
                        new HeapPriorityQueueSetFactory(keyGroupRange, 
numberOfKeyGroups, 128);
 
+               ReadableConfig config = 
env.getTaskManagerInfo().getConfiguration();
+               int fsReadBufferSize = 
config.get(CheckpointingOptions.FS_READ_BUFFER_SIZE);

Review comment:
       done

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
##########
@@ -331,13 +331,18 @@ public OperatorStateBackend createOperatorStateBackend(
                TaskStateManager taskStateManager = env.getTaskStateManager();
                HeapPriorityQueueSetFactory priorityQueueSetFactory =
                        new HeapPriorityQueueSetFactory(keyGroupRange, 
numberOfKeyGroups, 128);
+
+               ReadableConfig config = 
env.getTaskManagerInfo().getConfiguration();
+               int fsReadBufferSize = 
config.get(CheckpointingOptions.FS_READ_BUFFER_SIZE);

Review comment:
       done

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
##########
@@ -134,13 +136,18 @@ public CheckpointStreamFactory 
resolveCheckpointStorageLocation(long checkpointI
                MetricGroup metricGroup,
                @Nonnull Collection<KeyedStateHandle> stateHandles,
                CloseableRegistry cancelStreamRegistry) {
+
+               Configuration configuration = 
env.getTaskManagerInfo().getConfiguration();
+               int fsReadBufferSize = 
configuration.getInteger(CheckpointingOptions.FS_READ_BUFFER_SIZE);

Review comment:
       done

##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
##########
@@ -540,6 +540,10 @@ public CheckpointStorageAccess 
createCheckpointStorage(JobID jobId) throws IOExc
 
                ExecutionConfig executionConfig = env.getExecutionConfig();
                StreamCompressionDecorator keyGroupCompressionDecorator = 
getCompressionDecorator(executionConfig);
+
+               ReadableConfig config = 
env.getTaskManagerInfo().getConfiguration();
+               int fsReadBufferSize = 
config.get(CheckpointingOptions.FS_READ_BUFFER_SIZE);

Review comment:
       done

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FSDataBufferedInputStreamTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
+/**
+ * Unit tests for the {@link FSDataBufferedInputStream}.
+ */
+public class FSDataBufferedInputStreamTest {
+
+       private static final int KB = 1024;
+       private static final int MB = 1024 * KB;
+
+       FsCheckpointStreamFactory.FsCheckpointStateOutputStream outputStream;
+       DataOutputViewStreamWrapper outView;
+
+       @Rule
+       public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+       @Before
+       public void before() throws IOException {
+               String path = temporaryFolder.newFolder().getAbsolutePath();
+               File outFile = new File(path, 
String.valueOf(UUID.randomUUID()));
+
+               Path outPath = new Path(outFile.toURI());
+               outputStream = new 
FsCheckpointStreamFactory.FsCheckpointStateOutputStream(
+                       outPath, outPath.getFileSystem(), MB, KB);
+               outView = new DataOutputViewStreamWrapper(outputStream);
+       }
+
+       @Test
+       public void testOrderRead() throws Exception {
+               // write
+               int num = 10_000_000;
+               StreamStateHandle stateHandle = writeData(num);
+               Assert.assertNotNull(stateHandle);
+
+               int expectedStateSize = 2 * num * Integer.BYTES;
+               Assert.assertEquals(expectedStateSize, 
stateHandle.getStateSize());
+
+               // start read
+               FSDataInputStream fsDataInputStream = 
stateHandle.openInputStream();
+               FSDataBufferedInputStream bufferedInputStream = new 
FSDataBufferedInputStream(fsDataInputStream);
+               DataInputViewStreamWrapper inputView = new 
DataInputViewStreamWrapper(bufferedInputStream);
+
+               for (int i = 1; i <= num; i++) {
+                       int read = inputView.readInt();
+                       Assert.assertEquals(i, read);
+               }
+               for (int i = num; i >= 1; i--) {
+                       int read = inputView.readInt();
+                       Assert.assertEquals(i, read);
+               }
+               bufferedInputStream.close();
+               stateHandle.discardState();
+       }
+
+       @Test
+       public void testSeek() throws Exception {

Review comment:
       @klion26 , thanks for your suggest.
   I have finished it. I hope you can help review again when you are free.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to