flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From srich...@apache.org
Subject flink git commit: [FLINK-8927][checkpointing] Eagerly release the checkpoint object in RocksDB incremental snapshots
Date Mon, 12 Mar 2018 21:48:53 GMT
Repository: flink
Updated Branches:
  refs/heads/release-1.5 84a202f15 -> 69377cdc3


[FLINK-8927][checkpointing] Eagerly release the checkpoint object in RocksDB incremental snapshots

This closes #5682.

(cherry picked from commit 3debf47)


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/69377cdc
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/69377cdc
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/69377cdc

Branch: refs/heads/release-1.5
Commit: 69377cdc32d1b91b5dd1850afa550eaff808e7c8
Parents: 84a202f
Author: sihuazhou <summerleafs@163.com>
Authored: Mon Mar 12 15:04:57 2018 +0100
Committer: Stefan Richter <s.richter@data-artisans.com>
Committed: Mon Mar 12 22:48:43 2018 +0100

----------------------------------------------------------------------
 .../flink/contrib/streaming/state/RocksDBKeyedStateBackend.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/69377cdc/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 8f95b18..6a66121 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -2318,8 +2318,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K>
{
 			}
 
 			// create hard links of living files in the snapshot path
-			Checkpoint checkpoint = Checkpoint.create(stateBackend.db);
-			checkpoint.createCheckpoint(localBackupDirectory.getDirectory().getPath());
+			try (Checkpoint checkpoint = Checkpoint.create(stateBackend.db)) {
+				checkpoint.createCheckpoint(localBackupDirectory.getDirectory().getPath());
+			}
 		}
 
 		@Nonnull


Mime
View raw message