[FLINK-35803][checkpoint] Fix the wrong file reuse in cp file-merging (#25069)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/PhysicalFile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/PhysicalFile.java
index 6f3f1b5..5e68ca8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/PhysicalFile.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/PhysicalFile.java
@@ -121,7 +121,7 @@
         this.scope = scope;
         this.size = new AtomicLong(0);
         this.dataSize = new AtomicLong(0);
-        this.couldReuse = true;
+        this.couldReuse = owned;
         this.logicalFileRefCount = new AtomicInteger(0);
         this.isOwned = owned;
     }