fredia commented on code in PR #23514: URL: https://github.com/apache/flink/pull/23514#discussion_r1366385484
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileMergingCheckpointStateOutputStream.java: ########## @@ -0,0 +1,255 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager; +import org.apache.flink.runtime.checkpoint.filemerging.SegmentFileStateHandle; +import org.apache.flink.runtime.state.CheckpointStateOutputStream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A {@link CheckpointStateOutputStream} that writes into a segment of a file and returns a {@link + * SegmentFileStateHandle} upon closing. Multiple {@link FileMergingCheckpointStateOutputStream} + * objects can reuse the same underlying file, so that the checkpoint files are merged. + * + * <p><strong>Important</strong>: This implementation is NOT thread-safe. Multiple data streams + * multiplexing the same file should NOT write concurrently. Instead, it is expected that only after + * one data stream is closed, will other data streams reuse and write to the same underlying file. + */ +public class FileMergingCheckpointStateOutputStream extends CheckpointStateOutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(FileMergingCheckpointStateOutputStream.class); + + /** + * A proxy of the {@link FileMergingSnapshotManager} that owns this {@link + * FileMergingCheckpointStateOutputStream}, with the interfaces for dealing with physical files. + */ + public abstract static class FileMergingSnapshotManagerProxy { + /** + * Provide a physical file. + * + * @return Output stream and path of the physical file. + * @throws IOException if the physical file cannot be created or opened. + */ + public abstract Tuple2<FSDataOutputStream, Path> providePhysicalFile() throws IOException; + + /** + * Close the stream and create a {@link SegmentFileStateHandle} for a file segment. + * + * @param filePath Path of the physical file. + * @param startPos Start position of the segment in the physical file. + * @param stateSize Size of the segment. + * @return The state handle of the segment. + * @throws IOException if any exception happens when closing the file. + */ + public abstract SegmentFileStateHandle closeStreamAndCreateStateHandle( + Path filePath, long startPos, long stateSize) throws IOException; + + /** + * Notify the {@link FileMergingSnapshotManager} that the stream is closed exceptionally. + * + * @throws IOException if any exception happens when deleting the file. + */ + public abstract void closeStreamExceptionally() throws IOException; + } + + private final FileMergingSnapshotManagerProxy fileMergingSnapshotManagerProxy; + + private volatile boolean closed; + + /** path of the underlying physical file. */ + private Path filePath; + + /** the stream that writes to the underlying physical file. */ + private @Nullable FSDataOutputStream outputStream; + + /** start position in the physical file. */ + private long startPos; + + /** current position relative to startPos. */ + long curPosRelative = 0; + + /** the buffer for writing to the physical file. */ + private final byte[] writeBuffer; + + /** current position in the writeBuffer. */ + int bufferPos; + + public FileMergingCheckpointStateOutputStream( + long checkpointID, + int bufferSize, + FileMergingSnapshotManagerProxy fileMergingSnapshotManagerProxy) { + this.fileMergingSnapshotManagerProxy = fileMergingSnapshotManagerProxy; + this.writeBuffer = new byte[bufferSize]; + } + + /** Assign a physical file to this stream and initialize the outputStream. */ + private void initializeOutputStream() throws IOException { + Tuple2<FSDataOutputStream, Path> streamAndPath = + fileMergingSnapshotManagerProxy.providePhysicalFile(); + outputStream = streamAndPath.f0; + startPos = outputStream.getPos(); + filePath = streamAndPath.f1; + } + + @Override + public long getPos() throws IOException { + // The starting position is not determined until a physical file has been assigned, so + // we return the relative value to the starting position in this method + return bufferPos + curPosRelative; + } + + @Override + public void write(int b) throws IOException { + if (bufferPos >= writeBuffer.length) { + flushToFile(); + } + writeBuffer[bufferPos++] = (byte) b; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (len < writeBuffer.length) { + // copy it into our write buffer first + final int remaining = writeBuffer.length - bufferPos; + if (len > remaining) { + // copy as much as fits + System.arraycopy(b, off, writeBuffer, bufferPos, remaining); + off += remaining; + len -= remaining; + bufferPos += remaining; + + // flushToFile the write buffer to make it clear again + flushToFile(); + } + + // copy what is in the buffer + System.arraycopy(b, off, writeBuffer, bufferPos, len); + bufferPos += len; + } else { + // flushToFile the current buffer, outputStream is initialized if it is null + flushToFile(); + // write the bytes directly + outputStream.write(b, off, len); + curPosRelative += len; + } + } + + @Override + public void flush() throws IOException { + if (outputStream != null) { + flushToFile(); + } + } + + @Override + public void sync() throws IOException { + if (outputStream != null) { + outputStream.sync(); + } + } + + @Nullable + @Override + public SegmentFileStateHandle closeAndGetHandle() throws IOException { + // check if there was nothing ever written + if (outputStream == null && bufferPos == 0) { + return null; + } + + synchronized (this) { + if (!closed) { + try { + flushToFile(); + + bufferPos = writeBuffer.length; + + return fileMergingSnapshotManagerProxy.closeStreamAndCreateStateHandle( + filePath, startPos, curPosRelative); + } catch (Exception exception) { + fileMergingSnapshotManagerProxy.closeStreamExceptionally(); + + throw new IOException( + "Could not flush to file and close the file system " + + "output stream to " + + filePath + + " in order to obtain the " + + "stream state handle", + exception); + } finally { + closed = true; + } + } else { + throw new IOException("Stream has already been closed and discarded."); + } + } + } + + /** + * This method throws no exception if the close fails, but only logs the error. This is to be + * consistent with {@link FsCheckpointStreamFactory.FsCheckpointStateOutputStream#close()}. + */ + @Override + public void close() { + if (!closed) { + closed = true; + + // make sure write requests need to go to 'flushToFile()' where they recognized + // that the stream is closed + bufferPos = writeBuffer.length; + try { + fileMergingSnapshotManagerProxy.closeStreamExceptionally(); Review Comment: Are all `close()` calls exceptions? How about renaming `closeStreamExceptionally()`? ########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java: ########## @@ -206,6 +213,64 @@ protected PhysicalFile createPhysicalFile(SubtaskKey subtaskKey, CheckpointedSta "Could not open output stream for state file merging.", latestException); } + @Override + public FileMergingCheckpointStateOutputStream createCheckpointStateOutputStream( Review Comment: Where `FileMergingCheckpointStateOutputStream` will be used? I guess we should add `createTaskOwnedStateStream()` in `FsMergingCheckpointStorageAccess`, and call `createCheckpointStateOutputStream()`, WDYT? ########## flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerTest.java: ########## @@ -18,19 +18,28 @@ package org.apache.flink.runtime.checkpoint.filemerging; import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointedStateScope; import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess; +import org.apache.flink.runtime.state.filesystem.FileMergingCheckpointStateOutputStream; +import org.junit.jupiter.api.Assertions; Review Comment: It‘s better to use AssertJ instead of Assertions. https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#:~:text=Testing-,%23,-Tooling ########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/SegmentFileStateHandle.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.checkpoint.filemerging; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.runtime.state.filesystem.FsSegmentDataInputStream; + +import java.io.IOException; +import java.util.Objects; + +/** + * {@link FileStateHandle} for state that was written to a file segment. A {@link + * SegmentFileStateHandle} represents a {@link LogicalFile}, which has already been written to a + * segment in a physical file. + * + * <p>TODO (FLINK-32079): serialization and deserialization of {@link SegmentFileStateHandle}. + */ +public class SegmentFileStateHandle extends FileStateHandle { + + private static final long serialVersionUID = 1L; + + private final long startPos; + + private final CheckpointedStateScope scope; + + /** + * Creates a new segment file state for the given file path. + * + * @param filePath The path to the file that stores the state. + * @param startPos Start position of the segment in the physical file. + * @param stateSize Size of the segment. + * @param scope The state's scope, whether it is exclusive or shared. + */ + public SegmentFileStateHandle( + Path filePath, long startPos, long stateSize, CheckpointedStateScope scope) { + super(filePath, stateSize); + this.startPos = startPos; + this.scope = scope; + } + + /** + * This method should be empty, so that JM is not in charge of the lifecycle of files in a + * file-merging checkpoint. + */ + @Override + public void discardState() {} + + @Override + public FSDataInputStream openInputStream() throws IOException { + FSDataInputStream inputStream = super.openInputStream(); + return new FsSegmentDataInputStream(inputStream, startPos, stateSize); + } + + public long getStartPos() { + return startPos; + } + + @Override + public long getStateSize() { + return stateSize; + } + + public long getEndPos() { Review Comment: Looks like it's not used by others. ########## flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerTest.java: ########## @@ -18,19 +18,28 @@ package org.apache.flink.runtime.checkpoint.filemerging; import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointedStateScope; import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess; +import org.apache.flink.runtime.state.filesystem.FileMergingCheckpointStateOutputStream; +import org.junit.jupiter.api.Assertions; Review Comment: It‘s better to use AssertJ instead of Assertions. https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#:~:text=Testing-,%23,-Tooling ########## flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerTest.java: ########## @@ -249,6 +261,104 @@ void testSizeStatsInPhysicalFile() throws IOException { } } + @Test + public void testReusedFileWriting() throws Exception { + long checkpointId = 1; + int streamNum = 10; + int perStreamWriteNum = 128; + + // write random bytes and then read them from the file + byte[] bytes = new byte[streamNum * perStreamWriteNum]; + Random rd = new Random(); + rd.nextBytes(bytes); + int byteIndex = 0; + + SegmentFileStateHandle[] handles = new SegmentFileStateHandle[streamNum]; + try (FileMergingSnapshotManager fmsm = createFileMergingSnapshotManager(checkpointBaseDir); + CloseableRegistry closeableRegistry = new CloseableRegistry()) { + + // repeatedly get-write-close streams + for (int i = 0; i < streamNum; i++) { + FileMergingCheckpointStateOutputStream stream = + fmsm.createCheckpointStateOutputStream( + subtaskKey1, checkpointId, CheckpointedStateScope.EXCLUSIVE); + try { + closeableRegistry.registerCloseable(stream); + for (int j = 0; j < perStreamWriteNum; j++) { + stream.write(bytes[byteIndex++]); + } + handles[i] = stream.closeAndGetHandle(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + // assert the streams writes to the same file correctly + byteIndex = 0; + Path filePath = null; + for (SegmentFileStateHandle handle : handles) { + // check file path + Path thisFilePath = handle.getFilePath(); + Assertions.assertTrue(filePath == null || filePath.equals(thisFilePath)); + filePath = thisFilePath; + // check file content + FSDataInputStream is = handle.openInputStream(); + + closeableRegistry.registerCloseable(is); + int readValue; + + while ((readValue = is.read()) != -1) { + assertThat((byte) readValue).isEqualTo(bytes[byteIndex++]); + } + } + } + } + + @Test + public void testConcurrentWriting() throws Exception { Review Comment: I noticed that `FileMergingCheckpointStateOutputStream` is not thread-safe, how does this test guarantee thread safe? -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org