| commit | 2cad54846510db63254c503e6d9d44b6b5ddc124 | [log] [tgz] |
|---|---|---|
| author | Zakelly <zakelly.lan@gmail.com> | Thu Jul 11 10:30:28 2024 +0800 |
| committer | GitHub <noreply@github.com> | Thu Jul 11 10:30:28 2024 +0800 |
| tree | 02830d9f1f763fd660f3342f94bc45d1ed5e9dda | |
| parent | d04c70fae033bb04ed7a3bb69832be55b4425700 [diff] |
[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; }