Zakelly commented on code in PR #22590:
URL: https://github.com/apache/flink/pull/22590#discussion_r1209925907


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/PhysicalFile.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointedStateScope;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+/** An abstraction of physical files in file-merging checkpoints. */
+public class PhysicalFile {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(PhysicalFile.class);
+
+    /** Functional interface to delete the physical file. */
+    @FunctionalInterface
+    public interface PhysicalFileDeleter {
+        /** Close the outputStream if presented, and delete the file. */
+        void perform(@Nullable FSDataOutputStream outputStream, Path filePath);
+    }
+
+    /** Output stream to the file. It can be null if the file is closed. */
+    @Nullable private FSDataOutputStream outputStream;
+
+    private final AtomicInteger logicalFileRefCount;
+
+    private final AtomicLong size;
+
+    @Nullable private final PhysicalFileDeleter deleter;
+
+    private final Path filePath;
+
+    private final CheckpointedStateScope scope;
+
+    /**
+     * If a physical file is closed, it means no more file segments will be 
written to the physical
+     * file, and it can be deleted once its logicalFileRefCount decreases to 0.
+     */
+    private boolean closed;
+
+    /**
+     * A file can be deleted if: 1. It is closed, and 2. No more {@link 
LogicalFile}s have reference
+     * on it.
+     */
+    private boolean deleted = false;
+
+    public PhysicalFile(
+            @Nullable FSDataOutputStream outputStream,
+            Path filePath,
+            PhysicalFileDeleter deleter,
+            CheckpointedStateScope scope) {
+        this.filePath = filePath;
+        this.outputStream = outputStream;
+        this.closed = outputStream == null;
+        this.deleter = deleter;
+        this.scope = scope;
+        this.size = new AtomicLong(0);
+        this.logicalFileRefCount = new AtomicInteger(0);
+    }
+
+    @Nullable
+    public FSDataOutputStream getOutputStream() {
+        return outputStream;
+    }
+
+    void incRefCount() {
+        int newValue = this.logicalFileRefCount.incrementAndGet();
+        LOG.trace(
+                "Increase the reference count of physical file: {} by 1. New 
value is: {}.",
+                this.filePath,
+                newValue);
+    }
+
+    void decRefCount() throws IOException {
+        Preconditions.checkArgument(logicalFileRefCount.get() > 0);
+        int newValue = this.logicalFileRefCount.decrementAndGet();
+        LOG.trace(
+                "Decrease the reference count of physical file: {} by 1. New 
value is: {}. ",
+                this.filePath,
+                newValue);
+        deleteIfNecessary(false);
+    }
+
+    public void deleteIfNecessary(boolean forceClose) throws IOException {
+        if (forceClose) {
+            this.close();
+        }
+
+        synchronized (this) {
+            if (!isOpen() && !deleted && this.logicalFileRefCount.get() <= 0) {
+                if (deleter != null) {
+                    deleter.perform(outputStream, filePath);
+                }
+                this.deleted = true;
+            }
+        }

Review Comment:
   I removed the `forceClose` variable since it is useless currently.



-- 
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: [email protected]

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

Reply via email to