From 10b1bfe6f1840e417468878ce7d63e59a873b939 Mon Sep 17 00:00:00 2001 From: liningrui Date: Mon, 15 Mar 2021 16:43:08 +0800 Subject: [PATCH 01/13] Let snapshot saved disk same as rocksdb data path * The goal is to ensure that snapshots can be generated in hard link Change-Id: Idfa9387e46ac094a9c4cd69fee310b271127d889 --- .../hugegraph/api/profile/GraphsAPI.java | 14 ++ .../hugegraph/auth/HugeGraphAuthProxy.java | 6 + .../java/com/baidu/hugegraph/HugeGraph.java | 2 + .../baidu/hugegraph/StandardHugeGraph.java | 10 ++ .../store/AbstractBackendStoreProvider.java | 7 +- .../hugegraph/backend/store/BackendStore.java | 5 +- .../backend/store/raft/StoreSnapshotFile.java | 169 +++++++++++++----- .../backend/store/raft/StoreStateMachine.java | 4 +- .../baidu/hugegraph/util/CompressUtil.java | 8 +- .../store/rocksdb/RocksDBStdSessions.java | 11 +- .../backend/store/rocksdb/RocksDBStore.java | 98 ++++++---- 11 files changed, 243 insertions(+), 91 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java index 970edd3420..3557553ab1 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java @@ -138,6 +138,20 @@ public void clear(@Context GraphManager manager, g.truncateBackend(); } + @GET + @Timed + @Path("{name}/snapshot") + @Produces(APPLICATION_JSON_WITH_CHARSET) + @RolesAllowed("admin") + public Object snapshot(@Context GraphManager manager, + @PathParam("name") String name) { + LOG.debug("Generate snapshot for graph '{}'", name); + + HugeGraph g = graph(manager, name); + g.writeSnapshot(); + return ImmutableMap.of(name, "succeed"); + } + @PUT @Timed @Path("{name}/mode") diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java index bb495d7619..5f1dba81fe 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java @@ -700,6 +700,12 @@ public void truncateBackend() { } } + @Override + public void writeSnapshot() { + verifyAdminPermission(); + this.hugegraph.writeSnapshot(); + } + private void verifyAdminPermission() { verifyPermission(HugePermission.ANY, ResourceType.ROOT); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java index f9dd690144..b3dd372358 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java @@ -153,6 +153,8 @@ public interface HugeGraph extends Graph { public void clearBackend(); public void truncateBackend(); + public void writeSnapshot(); + @Override public HugeFeatures features(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index b6d4821119..40f352c717 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -368,6 +368,16 @@ public void truncateBackend() { LOG.info("Graph '{}' has been truncated", this.name); } + public void writeSnapshot() { + LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); + try { + this.storeProvider.writeSnapshot(); + } finally { + LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); + } + LOG.info("Graph '{}' has writed snapshot", this.name); + } + private SchemaTransaction openSchemaTransaction() throws HugeException { this.checkGraphNotClosed(); try { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index 7f5a44920c..5fe09aa593 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -152,12 +152,15 @@ public void initSystemInfo(HugeGraph graph) { @Override public void writeSnapshot() { - // TODO: to be implemented + String snapshotPrefix = "snapshot"; + for (BackendStore store : this.stores.values()) { + store.writeSnapshot(snapshotPrefix); + } } @Override public void readSnapshot() { - // TODO: to be implemented + throw new UnsupportedOperationException("readSnapshot"); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java index 209ed10cf7..2ce69f43c1 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java @@ -20,6 +20,7 @@ package com.baidu.hugegraph.backend.store; import java.util.Iterator; +import java.util.Set; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.id.IdGenerator; @@ -116,11 +117,11 @@ public default void setCounterLowest(HugeType type, long lowest) { // Get current counter for a specific type public long getCounter(HugeType type); - public default void writeSnapshot(String snapshotPath) { + public default Set writeSnapshot(String snapshotDir) { throw new UnsupportedOperationException("writeSnapshot"); } - public default void readSnapshot(String snapshotPath) { + public default void readSnapshot(String snapshotDir) { throw new UnsupportedOperationException("readSnapshot"); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index ff1a9f4a21..a2f469570f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -24,11 +24,14 @@ import java.io.File; import java.io.IOException; import java.nio.file.Paths; -import java.util.concurrent.CompletableFuture; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.zip.Checksum; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import com.alipay.sofa.jraft.Closure; @@ -45,8 +48,10 @@ public class StoreSnapshotFile { private static final Logger LOG = Log.logger(StoreSnapshotFile.class); - private static final String SNAPSHOT_DIR = "ss"; - private static final String SNAPSHOT_ARCHIVE = "ss.zip"; + private static final String SNAPSHOT_DIR = "snapshot"; + private static final String ARCHIVE_FORMAT = ".tar"; + private static final String SNAPSHOT_ARCHIVE = SNAPSHOT_DIR + ARCHIVE_FORMAT; + private static final String MANIFEST = "manifest"; private final RaftBackendStore[] stores; @@ -56,27 +61,22 @@ public StoreSnapshotFile(RaftBackendStore[] stores) { public void save(SnapshotWriter writer, Closure done, ExecutorService executor) { - String writerPath = writer.getPath(); - String snapshotPath = Paths.get(writerPath, SNAPSHOT_DIR).toString(); try { - this.doSnapshotSave(snapshotPath).whenComplete((metaBuilder, t) -> { - if (t == null) { - executor.execute(() -> compressSnapshot(writer, metaBuilder, - done)); - } else { - LOG.error("Failed to save snapshot, path={}, files={}", - writerPath, writer.listFiles(), t); - done.run(new Status(RaftError.EIO, - "Failed to save snapshot at %s, error is %s", - writerPath, t.getMessage())); - } + // Write snapshot to real directory + Set snapshotDirs = this.doSnapshotSave(); + executor.execute(() -> { + Set tarSnapshotFiles = + this.compressSnapshotDir(snapshotDirs, done); + String jraftSnapshotPath = + this.writeManifest(writer, tarSnapshotFiles, done); + this.deleteSnapshotDir(snapshotDirs, done); + this.compressJraftSnapshotDir(writer, jraftSnapshotPath, done); }); } catch (Throwable t) { - LOG.error("Failed to save snapshot, path={}, files={}, {}.", - writerPath, writer.listFiles(), t); + LOG.error("Failed to save snapshot", t); done.run(new Status(RaftError.EIO, - "Failed to save snapshot at %s, error is %s", - writerPath, t.getMessage())); + "Failed to save snapshot, error is %s", + t.getMessage())); } } @@ -84,14 +84,19 @@ public boolean load(SnapshotReader reader) { LocalFileMeta meta = (LocalFileMeta) reader.getFileMeta(SNAPSHOT_ARCHIVE); String readerPath = reader.getPath(); if (meta == null) { - LOG.error("Can't find snapshot archive file, path={}.", readerPath); + LOG.error("Can't find snapshot archive file, path={}", readerPath); return false; } - String snapshotPath = Paths.get(readerPath, SNAPSHOT_DIR).toString(); + String jraftSnapshotPath = Paths.get(readerPath, SNAPSHOT_DIR) + .toString(); try { - this.decompressSnapshot(readerPath, meta); - this.doSnapshotLoad(snapshotPath); - File tmp = new File(snapshotPath); + // decompress manifest and data directory + Set snapshotDirs = this.decompressSnapshot(readerPath, + jraftSnapshotPath, + meta); + this.doSnapshotLoad(); + this.deleteSnapshotDir(snapshotDirs); + File tmp = new File(jraftSnapshotPath); // Delete the decompressed temporary file. If the deletion fails // (although it is a small probability event), it may affect the // next snapshot decompression result. Therefore, the safest way @@ -102,39 +107,93 @@ public boolean load(SnapshotReader reader) { } return true; } catch (Throwable t) { - LOG.error("Failed to load snapshot, path={}, file list={}, {}.", - readerPath, reader.listFiles(), t); + LOG.error("Failed to load snapshot", t); return false; } } - private CompletableFuture doSnapshotSave( - String snapshotPath) { + private Set doSnapshotSave() { + Set snapshotDirs = new HashSet<>(); for (RaftBackendStore store : this.stores) { - String parentPath = Paths.get(snapshotPath, store.store()) - .toString(); - store.originStore().writeSnapshot(parentPath); + snapshotDirs.addAll(store.originStore().writeSnapshot(SNAPSHOT_DIR)); } - return CompletableFuture.completedFuture(LocalFileMeta.newBuilder()); + LOG.info("All snapshot dirs: {}", snapshotDirs); + return snapshotDirs; } - private void doSnapshotLoad(String snapshotPath) { + private void doSnapshotLoad() { for (RaftBackendStore store : this.stores) { - String parentPath = Paths.get(snapshotPath, store.store()) - .toString(); - store.originStore().readSnapshot(parentPath); + store.originStore().readSnapshot(SNAPSHOT_DIR); + } + } + + private Set compressSnapshotDir(Set snapshotDirs, + Closure done) { + // Compress all backend snapshot dir + Set tarSnapshotFiles = new HashSet<>(); + for (String snapshotDir : snapshotDirs) { + String outputFile = snapshotDir + ARCHIVE_FORMAT; + try { + CompressUtil.compressTar(snapshotDir, outputFile, new CRC64()); + } catch (IOException e) { + done.run(new Status(RaftError.EIO, + "Failed to compress backend snapshot dir " + + snapshotDir)); + } + tarSnapshotFiles.add(outputFile); + } + return tarSnapshotFiles; + } + + private void deleteSnapshotDir(Set snapshotDirs, + Closure done) { + // Delete all backend snapshot dir + for (String snapshotDir : snapshotDirs) { + try { + FileUtils.deleteDirectory(new File(snapshotDir)); + } catch (IOException e) { + done.run(new Status(RaftError.EIO, + "Failed to delete backend snapshot dir " + + snapshotDir)); + } + } + } + + private void deleteSnapshotDir(Set snapshotDirs) + throws IOException { + // Delete all backend snapshot dir + for (String snapshotDir : snapshotDirs) { + FileUtils.deleteDirectory(new File(snapshotDir)); + } + } + + private String writeManifest(SnapshotWriter writer, + Set tarSnapshotFiles, + Closure done) { + String writerPath = writer.getPath(); + // Write all backend compressed snapshot file path to manifest + String jraftSnapshotPath = Paths.get(writerPath, SNAPSHOT_DIR) + .toString(); + File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); + try { + FileUtils.writeLines(snapshotManifest, tarSnapshotFiles); + } catch (IOException e) { + done.run(new Status(RaftError.EIO, + "Failed to write backend snapshot file path " + + "to manifest")); } + return jraftSnapshotPath; } - private void compressSnapshot(SnapshotWriter writer, - LocalFileMeta.Builder metaBuilder, - Closure done) { + private void compressJraftSnapshotDir(SnapshotWriter writer, + String jraftSnapshotPath, + Closure done) { String writerPath = writer.getPath(); String outputFile = Paths.get(writerPath, SNAPSHOT_ARCHIVE).toString(); try { + LocalFileMeta.Builder metaBuilder = LocalFileMeta.newBuilder(); Checksum checksum = new CRC64(); - CompressUtil.compressTar(writerPath, SNAPSHOT_DIR, - outputFile, checksum); + CompressUtil.compressTar(jraftSnapshotPath, outputFile, checksum); metaBuilder.setChecksum(Long.toHexString(checksum.getValue())); if (writer.addFile(SNAPSHOT_ARCHIVE, metaBuilder.build())) { done.run(Status.OK()); @@ -152,15 +211,33 @@ private void compressSnapshot(SnapshotWriter writer, } } - private void decompressSnapshot(String readerPath, LocalFileMeta meta) - throws IOException { - String sourceFile = Paths.get(readerPath, SNAPSHOT_ARCHIVE).toString(); + private Set decompressSnapshot(String readerPath, + String jraftSnapshotPath, + LocalFileMeta meta) + throws IOException { + String archiveFile = Paths.get(readerPath, SNAPSHOT_ARCHIVE).toString(); Checksum checksum = new CRC64(); - CompressUtil.decompressTar(sourceFile, readerPath, checksum); + CompressUtil.decompressTar(archiveFile, readerPath, checksum); if (meta.hasChecksum()) { E.checkArgument(meta.getChecksum().equals( Long.toHexString(checksum.getValue())), "Snapshot checksum failed"); } + + File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); + List compressedSnapshotFiles = FileUtils.readLines( + snapshotManifest); + Set snapshotDirs = new HashSet<>(); + for (String compressedSnapshotFile : compressedSnapshotFiles) { + String targetDir = Paths.get(compressedSnapshotFile).getParent() + .toString(); + CompressUtil.decompressTar(compressedSnapshotFile, targetDir, + new CRC64()); + + String snapshotDir = StringUtils.removeEnd(compressedSnapshotFile, + ARCHIVE_FORMAT); + snapshotDirs.add(snapshotDir); + } + return snapshotDirs; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java index a5697209a6..12efc35682 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java @@ -229,8 +229,8 @@ public boolean onSnapshotLoad(SnapshotReader reader) { @Override public void onLeaderStart(long term) { - LOG.info("The node {} become to leader", this.node().nodeId()); - this.node().onLeaderInfoChange(this.node().nodeId(), true); + LOG.info("The node {} become to leader", this.context.endpoint()); + this.node().onLeaderInfoChange(this.context.endpoint(), true); super.onLeaderStart(term); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/CompressUtil.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/CompressUtil.java index 032f932ae7..c1903f069a 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/CompressUtil.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/CompressUtil.java @@ -62,6 +62,12 @@ public final class CompressUtil { public static void compressTar(String rootDir, String sourceDir, String outputFile, Checksum checksum) throws IOException { + Path source = Paths.get(rootDir, sourceDir); + compressTar(source.toString(), outputFile, checksum); + } + + public static void compressTar(String inputDir, String outputFile, + Checksum checksum) throws IOException { LZ4Factory factory = LZ4Factory.fastestInstance(); LZ4Compressor compressor = factory.fastCompressor(); int blockSize = RaftSharedContext.BLOCK_SIZE; @@ -72,7 +78,7 @@ public static void compressTar(String rootDir, String sourceDir, blockSize, compressor); TarArchiveOutputStream tos = new TarArchiveOutputStream(lz4os)) { - Path source = Paths.get(rootDir, sourceDir); + Path source = Paths.get(inputDir); CompressUtil.tarDir(source, tos); tos.flush(); fos.getFD().sync(); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 3bbfd9237d..48191965c8 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -59,6 +59,7 @@ import org.rocksdb.SstFileManager; import org.rocksdb.WriteBatch; import org.rocksdb.WriteOptions; +import org.slf4j.Logger; import com.baidu.hugegraph.backend.BackendException; import com.baidu.hugegraph.backend.serializer.BinarySerializer; @@ -69,12 +70,14 @@ import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.GZipUtil; +import com.baidu.hugegraph.util.Log; import com.baidu.hugegraph.util.StringEncoding; import com.google.common.collect.ImmutableList; public class RocksDBStdSessions extends RocksDBSessions { + private static final Logger LOG = Log.logger(RocksDBStdSessions.class); + private final HugeConfig config; private final String dataPath; private final String walPath; @@ -321,19 +324,19 @@ public RocksDBSessions copy(HugeConfig config, } @Override - public void createSnapshot(String parentPath) { - String md5 = GZipUtil.md5(this.dataPath); - String snapshotPath = Paths.get(parentPath, md5).toString(); + public void createSnapshot(String snapshotPath) { // https://github.com/facebook/rocksdb/wiki/Checkpoints try (Checkpoint checkpoint = Checkpoint.create(this.rocksdb)) { String tempPath = snapshotPath + "_temp"; File tempFile = new File(tempPath); FileUtils.deleteDirectory(tempFile); + LOG.debug("Deleted temp directory {}", tempFile); FileUtils.forceMkdir(tempFile.getParentFile()); checkpoint.createCheckpoint(tempPath); File snapshotFile = new File(snapshotPath); FileUtils.deleteDirectory(snapshotFile); + LOG.debug("Deleted stale directory {}", snapshotFile); if (!tempFile.renameTo(snapshotFile)) { throw new IOException(String.format("Failed to rename %s to %s", tempFile, snapshotFile)); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 635e33024a..3ec21d2661 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; @@ -43,7 +44,6 @@ import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.RocksDBException; import org.slf4j.Logger; @@ -116,7 +116,7 @@ private void registerMetaHandlers() { this.registerMetaHandler("metrics", (session, meta, args) -> { List dbs = new ArrayList<>(); dbs.add(this.sessions); - dbs.addAll(tableDBMapping().values()); + dbs.addAll(this.tableDBMapping().values()); RocksDBMetrics metrics = new RocksDBMetrics(dbs, session); return metrics.getMetrics(); @@ -200,7 +200,7 @@ public synchronized void open(HugeConfig config) { })); } } - waitOpenFinish(futures, openPool); + this.waitOpenFinish(futures, openPool); } private void waitOpenFinish(List> futures, @@ -349,7 +349,7 @@ protected Map tableDBMapping() { Map tableDBMap = InsertionOrderUtil.newMap(); for (Entry e : this.tableDiskMapping.entrySet()) { String table = this.table(e.getKey()).table(); - RocksDBSessions db = db(e.getValue()); + RocksDBSessions db = this.db(e.getValue()); tableDBMap.put(table, db); } return tableDBMap; @@ -539,7 +539,7 @@ public synchronized void truncate() { this.clear(false); this.init(); // clear write batch - dbs.values().forEach(BackendSessionPool::forceResetSessions); + this.dbs.values().forEach(BackendSessionPool::forceResetSessions); LOG.debug("Store truncated: {}", this.store); } finally { writeLock.unlock(); @@ -601,43 +601,72 @@ protected Session session(HugeType tableType) { // Optimized disk String disk = this.tableDiskMapping.get(tableType); if (disk != null) { - return db(disk).session(); + return this.db(disk).session(); } return this.sessions.session(); } @Override - public void writeSnapshot(String parentPath) { - Lock writeLock = this.storeLock.writeLock(); - writeLock.lock(); + public Set writeSnapshot(String snapshotPrefix) { + Lock readLock = this.storeLock.readLock(); + readLock.lock(); try { + Set uniqueParents = new HashSet<>(); // Every rocksdb instance should create an snapshot - for (RocksDBSessions sessions : this.sessions()) { - sessions.createSnapshot(parentPath); + for (Map.Entry entry : this.dbs.entrySet()) { + // Like: parent_path/rocksdb-data/m + // parent_path/rocksdb-vertex/g + Path originDataPath = Paths.get(entry.getKey()); + Path parentParentPath = originDataPath.getParent().getParent(); + // Like: rocksdb-data/m + // rocksdb-vertex/g + Path pureDataPath = parentParentPath.relativize(originDataPath); + // Like: parent_path/snapshot_rocksdb-data/m + // parent_path/snapshot_rocksdb-vertex/g + Path snapshotPath = parentParentPath.resolve(snapshotPrefix + + "_" + pureDataPath); + LOG.debug("The origin data path: {}", originDataPath); + LOG.debug("The snapshot data path: {}", snapshotPath); + RocksDBSessions sessions = entry.getValue(); + sessions.createSnapshot(snapshotPath.toString()); + + uniqueParents.add(snapshotPath.getParent().toString()); } + LOG.info("The store '{}' save snapshot successfully", this.store); + return uniqueParents; } finally { - writeLock.unlock(); + readLock.unlock(); } } @Override - public void readSnapshot(String parentPath) { - Lock writeLock = this.storeLock.writeLock(); - writeLock.lock(); + public void readSnapshot(String snapshotPrefix) { + Lock readLock = this.storeLock.readLock(); + readLock.lock(); try { if (!this.opened()) { return; } - - File[] snapshotFiles = new File(parentPath).listFiles(); - E.checkNotNull(snapshotFiles, "snapshot files"); - List> fileRenamePairs = new ArrayList<>(); - for (File snapshotFile : snapshotFiles) { - Session session = this.findMatchedSession(snapshotFile); - File dataFile = new File(session.dataPath()); - fileRenamePairs.add(Pair.of(snapshotFile, dataFile)); + Map fileNameMaps = new HashMap<>(); + for (Map.Entry entry : this.dbs.entrySet()) { + // Like: parent_path/rocksdb-data/m + // parent_path/rocksdb-vertex/g + Path originDataPath = Paths.get(entry.getKey()); + Path parentParentPath = originDataPath.getParent().getParent(); + // Like: rocksdb-data/m + // rocksdb-vertex/g + Path pureDataPath = parentParentPath.relativize(originDataPath); + // Like: parent_path/snapshot_rocksdb-data/m + // parent_path/snapshot_rocksdb-vertex/g + Path snapshotPath = parentParentPath.resolve(snapshotPrefix + + "_" + pureDataPath); + LOG.debug("The origin data path: {}", originDataPath); + LOG.debug("The snapshot data path: {}", snapshotPath); + fileNameMaps.put(originDataPath, snapshotPath); } + E.checkState(!fileNameMaps.isEmpty(), + "The file name maps shouldn't be empty"); /* * NOTE: must close rocksdb instance before deleting file directory, * if close after copying the snapshot directory to origin position, @@ -646,30 +675,31 @@ public void readSnapshot(String parentPath) { for (RocksDBSessions sessions : this.sessions()) { sessions.forceCloseRocksDB(); } - // Copy snapshot file to dest file - for (Pair pair : fileRenamePairs) { - File snapshotFile = pair.getLeft(); - File dataFile = pair.getRight(); + // Move snapshot file to origin data path + for (Map.Entry entry : fileNameMaps.entrySet()) { + File originDataDir = entry.getKey().toFile(); + File snapshotDir = entry.getValue().toFile(); try { - if (dataFile.exists()) { - LOG.warn("Delete origin data directory {}", dataFile); - FileUtils.deleteDirectory(dataFile); + if (originDataDir.exists()) { + LOG.info("Delete origin data directory {}", + originDataDir); + FileUtils.deleteDirectory(originDataDir); } - FileUtils.moveDirectory(snapshotFile, dataFile); + FileUtils.moveDirectory(snapshotDir, originDataDir); } catch (IOException e) { throw new BackendException("Failed to move %s to %s", - e, snapshotFile, dataFile); + e, snapshotDir, originDataDir); } } // Reload rocksdb instance for (RocksDBSessions sessions : this.sessions()) { sessions.reload(); } - LOG.info("The store {} load snapshot successfully", this.store); + LOG.info("The store '{}' load snapshot successfully", this.store); } catch (RocksDBException e) { throw new BackendException("Failed to reload rocksdb", e); } finally { - writeLock.unlock(); + readLock.unlock(); } } From de8b72baf2e098ef7f9b1ddbc2bd50e488ba2caf Mon Sep 17 00:00:00 2001 From: liningrui Date: Tue, 16 Mar 2021 14:54:56 +0800 Subject: [PATCH 02/13] tiny improve Change-Id: I829ad9eaaf399b5fc1b8d25a90b2d1dd88c54894 --- .../java/com/baidu/hugegraph/api/profile/GraphsAPI.java | 8 ++++---- .../java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java | 6 +++--- .../src/main/java/com/baidu/hugegraph/HugeGraph.java | 5 ++++- .../main/java/com/baidu/hugegraph/StandardHugeGraph.java | 2 +- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java index 3557553ab1..8778ade422 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java @@ -138,17 +138,17 @@ public void clear(@Context GraphManager manager, g.truncateBackend(); } - @GET + @PUT @Timed @Path("{name}/snapshot") @Produces(APPLICATION_JSON_WITH_CHARSET) - @RolesAllowed("admin") + @RolesAllowed({"admin", "$owner=$name"}) public Object snapshot(@Context GraphManager manager, @PathParam("name") String name) { - LOG.debug("Generate snapshot for graph '{}'", name); + LOG.debug("Create snapshot for graph '{}'", name); HugeGraph g = graph(manager, name); - g.writeSnapshot(); + g.createSnapshot(); return ImmutableMap.of(name, "succeed"); } diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java index 5f1dba81fe..7ea7296a10 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java @@ -701,9 +701,9 @@ public void truncateBackend() { } @Override - public void writeSnapshot() { - verifyAdminPermission(); - this.hugegraph.writeSnapshot(); + public void createSnapshot() { + this.verifyPermission(HugePermission.WRITE, ResourceType.STATUS); + this.hugegraph.createSnapshot(); } private void verifyAdminPermission() { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java index b3dd372358..595202a5b6 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java @@ -153,7 +153,10 @@ public interface HugeGraph extends Graph { public void clearBackend(); public void truncateBackend(); - public void writeSnapshot(); + public void createSnapshot(); + public default void resumeSnapshot() { + throw new UnsupportedOperationException("HugeGraph.resumeSnapshot"); + } @Override public HugeFeatures features(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index 40f352c717..92c9e2dbdd 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -368,7 +368,7 @@ public void truncateBackend() { LOG.info("Graph '{}' has been truncated", this.name); } - public void writeSnapshot() { + public void createSnapshot() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { this.storeProvider.writeSnapshot(); From bbb6faf7eed59151b51e13b402c83a5dd03b01c2 Mon Sep 17 00:00:00 2001 From: liningrui Date: Wed, 17 Mar 2021 11:56:27 +0800 Subject: [PATCH 03/13] Add resume snapshot API Change-Id: I2b212decb8f0639354e032a09e9ca40988b001da --- .../hugegraph/api/profile/GraphsAPI.java | 20 ++++++++++++++++--- .../hugegraph/auth/HugeGraphAuthProxy.java | 6 ++++++ .../java/com/baidu/hugegraph/HugeGraph.java | 4 +--- .../baidu/hugegraph/StandardHugeGraph.java | 13 +++++++++++- .../store/AbstractBackendStoreProvider.java | 8 ++++++-- .../store/raft/RaftBackendStoreProvider.java | 3 ++- .../backend/store/raft/StoreSnapshotFile.java | 2 +- .../backend/store/rocksdb/RocksDBStore.java | 2 ++ 8 files changed, 47 insertions(+), 11 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java index 8778ade422..5003001ca8 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java @@ -140,11 +140,11 @@ public void clear(@Context GraphManager manager, @PUT @Timed - @Path("{name}/snapshot") + @Path("{name}/create-snapshot") @Produces(APPLICATION_JSON_WITH_CHARSET) @RolesAllowed({"admin", "$owner=$name"}) - public Object snapshot(@Context GraphManager manager, - @PathParam("name") String name) { + public Object createSnapshot(@Context GraphManager manager, + @PathParam("name") String name) { LOG.debug("Create snapshot for graph '{}'", name); HugeGraph g = graph(manager, name); @@ -152,6 +152,20 @@ public Object snapshot(@Context GraphManager manager, return ImmutableMap.of(name, "succeed"); } + @PUT + @Timed + @Path("{name}/resume-snapshot") + @Produces(APPLICATION_JSON_WITH_CHARSET) + @RolesAllowed({"admin", "$owner=$name"}) + public Object resumeSnapshot(@Context GraphManager manager, + @PathParam("name") String name) { + LOG.debug("Resume snapshot for graph '{}'", name); + + HugeGraph g = graph(manager, name); + g.resumeSnapshot(); + return ImmutableMap.of(name, "succeed"); + } + @PUT @Timed @Path("{name}/mode") diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java index 7ea7296a10..34fcc6d495 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/auth/HugeGraphAuthProxy.java @@ -706,6 +706,12 @@ public void createSnapshot() { this.hugegraph.createSnapshot(); } + @Override + public void resumeSnapshot() { + this.verifyPermission(HugePermission.WRITE, ResourceType.STATUS); + this.hugegraph.resumeSnapshot(); + } + private void verifyAdminPermission() { verifyPermission(HugePermission.ANY, ResourceType.ROOT); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java index 595202a5b6..ed26bbb5d0 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java @@ -154,9 +154,7 @@ public interface HugeGraph extends Graph { public void truncateBackend(); public void createSnapshot(); - public default void resumeSnapshot() { - throw new UnsupportedOperationException("HugeGraph.resumeSnapshot"); - } + public void resumeSnapshot(); @Override public HugeFeatures features(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index 92c9e2dbdd..5e3b6febf5 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -375,7 +375,18 @@ public void createSnapshot() { } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); } - LOG.info("Graph '{}' has writed snapshot", this.name); + LOG.info("Graph '{}' has created snapshot", this.name); + } + + @Override + public void resumeSnapshot() { + LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); + try { + this.storeProvider.readSnapshot(); + } finally { + LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); + } + LOG.info("Graph '{}' has resumed from snapshot", this.name); } private SchemaTransaction openSchemaTransaction() throws HugeException { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index 5fe09aa593..be0d9d1948 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -27,6 +27,7 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.BackendException; +import com.baidu.hugegraph.backend.store.raft.StoreSnapshotFile; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.event.EventListener; import com.baidu.hugegraph.util.E; @@ -152,7 +153,7 @@ public void initSystemInfo(HugeGraph graph) { @Override public void writeSnapshot() { - String snapshotPrefix = "snapshot"; + String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; for (BackendStore store : this.stores.values()) { store.writeSnapshot(snapshotPrefix); } @@ -160,7 +161,10 @@ public void writeSnapshot() { @Override public void readSnapshot() { - throw new UnsupportedOperationException("readSnapshot"); + String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; + for (BackendStore store : this.stores.values()) { + store.readSnapshot(snapshotPrefix); + } } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java index 0e465da9ec..df049cf8dc 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java @@ -209,7 +209,8 @@ public void writeSnapshot() { @Override public void readSnapshot() { - // How to read snapshot by jraft explicity? + // jraft doesn't expose API to load snapshot + throw new UnsupportedOperationException(); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index a2f469570f..272ef89948 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -48,7 +48,7 @@ public class StoreSnapshotFile { private static final Logger LOG = Log.logger(StoreSnapshotFile.class); - private static final String SNAPSHOT_DIR = "snapshot"; + public static final String SNAPSHOT_DIR = "snapshot"; private static final String ARCHIVE_FORMAT = ".tar"; private static final String SNAPSHOT_ARCHIVE = SNAPSHOT_DIR + ARCHIVE_FORMAT; private static final String MANIFEST = "manifest"; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 3ec21d2661..baed8c88ef 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -686,6 +686,8 @@ public void readSnapshot(String snapshotPrefix) { FileUtils.deleteDirectory(originDataDir); } FileUtils.moveDirectory(snapshotDir, originDataDir); + LOG.info("Move snapshot directory {} to {}", + snapshotDir, originDataDir); } catch (IOException e) { throw new BackendException("Failed to move %s to %s", e, snapshotDir, originDataDir); From bce32ec0ceb30e64c4d2edba6b93cafe160666ad Mon Sep 17 00:00:00 2001 From: liningrui Date: Thu, 18 Mar 2021 17:01:09 +0800 Subject: [PATCH 04/13] tiny improve Change-Id: I67d88d798c6a16a66681296fbf29d73fb8bc356a --- .../store/AbstractBackendStoreProvider.java | 16 ++++++++++- .../hugegraph/backend/store/BackendStore.java | 2 +- .../backend/store/raft/StoreSnapshotFile.java | 27 +++---------------- .../backend/store/rocksdb/RocksDBStore.java | 15 ++++++++--- 4 files changed, 31 insertions(+), 29 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index be0d9d1948..918f4cdf91 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -19,10 +19,15 @@ package com.baidu.hugegraph.backend.store; +import java.io.File; +import java.io.IOException; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; +import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; @@ -162,8 +167,17 @@ public void writeSnapshot() { @Override public void readSnapshot() { String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; + Set snapshotDirs = new HashSet<>(); for (BackendStore store : this.stores.values()) { - store.readSnapshot(snapshotPrefix); + snapshotDirs.addAll(store.readSnapshot(snapshotPrefix)); + } + // Delete all snapshot parent directories + for (String snapshotDir : snapshotDirs) { + try { + FileUtils.deleteDirectory(new File(snapshotDir)); + } catch (IOException e) { + LOG.warn("Failed to delete snapshot directory {}", snapshotDir); + } } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java index 2ce69f43c1..914f3f5573 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java @@ -121,7 +121,7 @@ public default Set writeSnapshot(String snapshotDir) { throw new UnsupportedOperationException("writeSnapshot"); } - public default void readSnapshot(String snapshotDir) { + public default Set readSnapshot(String snapshotDir) { throw new UnsupportedOperationException("readSnapshot"); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index 272ef89948..f8ac2fe47a 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -31,7 +31,6 @@ import java.util.zip.Checksum; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import com.alipay.sofa.jraft.Closure; @@ -91,11 +90,8 @@ public boolean load(SnapshotReader reader) { .toString(); try { // decompress manifest and data directory - Set snapshotDirs = this.decompressSnapshot(readerPath, - jraftSnapshotPath, - meta); + this.decompressSnapshot(readerPath, jraftSnapshotPath, meta); this.doSnapshotLoad(); - this.deleteSnapshotDir(snapshotDirs); File tmp = new File(jraftSnapshotPath); // Delete the decompressed temporary file. If the deletion fails // (although it is a small probability event), it may affect the @@ -159,14 +155,6 @@ private void deleteSnapshotDir(Set snapshotDirs, } } - private void deleteSnapshotDir(Set snapshotDirs) - throws IOException { - // Delete all backend snapshot dir - for (String snapshotDir : snapshotDirs) { - FileUtils.deleteDirectory(new File(snapshotDir)); - } - } - private String writeManifest(SnapshotWriter writer, Set tarSnapshotFiles, Closure done) { @@ -211,10 +199,9 @@ private void compressJraftSnapshotDir(SnapshotWriter writer, } } - private Set decompressSnapshot(String readerPath, - String jraftSnapshotPath, - LocalFileMeta meta) - throws IOException { + private void decompressSnapshot(String readerPath, + String jraftSnapshotPath, + LocalFileMeta meta) throws IOException { String archiveFile = Paths.get(readerPath, SNAPSHOT_ARCHIVE).toString(); Checksum checksum = new CRC64(); CompressUtil.decompressTar(archiveFile, readerPath, checksum); @@ -227,17 +214,11 @@ private Set decompressSnapshot(String readerPath, File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); List compressedSnapshotFiles = FileUtils.readLines( snapshotManifest); - Set snapshotDirs = new HashSet<>(); for (String compressedSnapshotFile : compressedSnapshotFiles) { String targetDir = Paths.get(compressedSnapshotFile).getParent() .toString(); CompressUtil.decompressTar(compressedSnapshotFile, targetDir, new CRC64()); - - String snapshotDir = StringUtils.removeEnd(compressedSnapshotFile, - ARCHIVE_FORMAT); - snapshotDirs.add(snapshotDir); } - return snapshotDirs; } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index baed8c88ef..c3ed2528c7 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -617,7 +617,7 @@ public Set writeSnapshot(String snapshotPrefix) { for (Map.Entry entry : this.dbs.entrySet()) { // Like: parent_path/rocksdb-data/m // parent_path/rocksdb-vertex/g - Path originDataPath = Paths.get(entry.getKey()); + Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); Path parentParentPath = originDataPath.getParent().getParent(); // Like: rocksdb-data/m // rocksdb-vertex/g @@ -641,18 +641,20 @@ public Set writeSnapshot(String snapshotPrefix) { } @Override - public void readSnapshot(String snapshotPrefix) { + public Set readSnapshot(String snapshotPrefix) { Lock readLock = this.storeLock.readLock(); readLock.lock(); try { + Set uniqueParents = new HashSet<>(); if (!this.opened()) { - return; + return uniqueParents; } + Map fileNameMaps = new HashMap<>(); for (Map.Entry entry : this.dbs.entrySet()) { // Like: parent_path/rocksdb-data/m // parent_path/rocksdb-vertex/g - Path originDataPath = Paths.get(entry.getKey()); + Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); Path parentParentPath = originDataPath.getParent().getParent(); // Like: rocksdb-data/m // rocksdb-vertex/g @@ -661,9 +663,13 @@ public void readSnapshot(String snapshotPrefix) { // parent_path/snapshot_rocksdb-vertex/g Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + pureDataPath); + E.checkState(snapshotPath.toFile().exists(), + "The snapshot path '%s' doesn't exist"); LOG.debug("The origin data path: {}", originDataPath); LOG.debug("The snapshot data path: {}", snapshotPath); fileNameMaps.put(originDataPath, snapshotPath); + + uniqueParents.add(snapshotPath.getParent().toString()); } E.checkState(!fileNameMaps.isEmpty(), "The file name maps shouldn't be empty"); @@ -698,6 +704,7 @@ public void readSnapshot(String snapshotPrefix) { sessions.reload(); } LOG.info("The store '{}' load snapshot successfully", this.store); + return uniqueParents; } catch (RocksDBException e) { throw new BackendException("Failed to reload rocksdb", e); } finally { From 5adb2c6384214abe9ec579fd0270bea9a9674b28 Mon Sep 17 00:00:00 2001 From: liningrui Date: Fri, 19 Mar 2021 14:29:16 +0800 Subject: [PATCH 05/13] tiny improve Change-Id: I845cc78b20b94eee6bd2aff43038dd22c1a7834a --- .../baidu/hugegraph/StandardHugeGraph.java | 8 +++---- .../store/AbstractBackendStoreProvider.java | 12 ++++++---- .../hugegraph/backend/store/BackendStore.java | 4 ++-- .../backend/store/BackendStoreProvider.java | 4 ++-- .../store/raft/RaftBackendStoreProvider.java | 4 ++-- .../backend/store/raft/RaftSharedContext.java | 14 ++++++++++-- .../backend/store/raft/StoreSnapshotFile.java | 7 ++++-- .../backend/store/raft/StoreStateMachine.java | 15 +++---------- .../security/HugeSecurityManager.java | 22 ++++++++++++++----- .../backend/store/rocksdb/RocksDBStore.java | 4 ++-- .../baidu/hugegraph/core/EdgeCoreTest.java | 6 ++++- 11 files changed, 62 insertions(+), 38 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index 5e3b6febf5..a627b73b79 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -360,7 +360,7 @@ public void truncateBackend() { * When restarting, load the snapshot first and then read backend, * will not encounter such an intermediate state. */ - this.storeProvider.writeSnapshot(); + this.storeProvider.createSnapshot(); } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); } @@ -371,7 +371,7 @@ public void truncateBackend() { public void createSnapshot() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { - this.storeProvider.writeSnapshot(); + this.storeProvider.createSnapshot(); } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); } @@ -382,7 +382,7 @@ public void createSnapshot() { public void resumeSnapshot() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { - this.storeProvider.readSnapshot(); + this.storeProvider.resumeSnapshot(); } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); } @@ -1445,7 +1445,7 @@ public void invalid2(HugeType type, Object[] ids) { @Override public void clear(HugeType type) { - this.hub.notify(Events.CACHE, Cache.ACTION_CLEAR, type, null); + this.hub.notify(Events.CACHE, Cache.ACTION_CLEAR, type); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index 918f4cdf91..48aeed09b8 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -157,19 +157,23 @@ public void initSystemInfo(HugeGraph graph) { } @Override - public void writeSnapshot() { + public void createSnapshot() { + long begin = System.currentTimeMillis(); String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; for (BackendStore store : this.stores.values()) { - store.writeSnapshot(snapshotPrefix); + store.createSnapshot(snapshotPrefix); } + + LOG.info("Create snapshot cost {}ms", + System.currentTimeMillis() - begin); } @Override - public void readSnapshot() { + public void resumeSnapshot() { String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; Set snapshotDirs = new HashSet<>(); for (BackendStore store : this.stores.values()) { - snapshotDirs.addAll(store.readSnapshot(snapshotPrefix)); + snapshotDirs.addAll(store.resumeSnapshot(snapshotPrefix)); } // Delete all snapshot parent directories for (String snapshotDir : snapshotDirs) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java index 914f3f5573..42257fcdbb 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java @@ -117,11 +117,11 @@ public default void setCounterLowest(HugeType type, long lowest) { // Get current counter for a specific type public long getCounter(HugeType type); - public default Set writeSnapshot(String snapshotDir) { + public default Set createSnapshot(String snapshotDir) { throw new UnsupportedOperationException("writeSnapshot"); } - public default Set readSnapshot(String snapshotDir) { + public default Set resumeSnapshot(String snapshotDir) { throw new UnsupportedOperationException("readSnapshot"); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java index e7f815b64e..12058d7b73 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java @@ -54,9 +54,9 @@ public interface BackendStoreProvider { public void initSystemInfo(HugeGraph graph); - public void writeSnapshot(); + public void createSnapshot(); - public void readSnapshot(); + public void resumeSnapshot(); public void listen(EventListener listener); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java index df049cf8dc..2f55018563 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java @@ -199,7 +199,7 @@ public void initSystemInfo(HugeGraph graph) { } @Override - public void writeSnapshot() { + public void createSnapshot() { StoreCommand command = new StoreCommand(StoreType.ALL, StoreAction.SNAPSHOT, null); StoreClosure closure = new StoreClosure(command); @@ -208,7 +208,7 @@ public void writeSnapshot() { } @Override - public void readSnapshot() { + public void resumeSnapshot() { // jraft doesn't expose API to load snapshot throw new UnsupportedOperationException(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftSharedContext.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftSharedContext.java index 5b2815fd08..e4c7f44385 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftSharedContext.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftSharedContext.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.Paths; +import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; @@ -43,6 +44,7 @@ import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraphParams; import com.baidu.hugegraph.backend.cache.Cache; +import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.raft.rpc.ListPeersProcessor; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreType; @@ -259,7 +261,7 @@ public void clearCache() { this.notifyCache(Cache.ACTION_CLEAR, HugeType.VERTEX, null); } - protected void notifyCache(String action, HugeType type, Object id) { + protected void notifyCache(String action, HugeType type, List ids) { EventHub eventHub; if (type.isGraph()) { eventHub = this.params.graphEventHub(); @@ -270,7 +272,15 @@ protected void notifyCache(String action, HugeType type, Object id) { } try { // How to avoid update cache from server info - eventHub.notify(Events.CACHE, action, type, id); + if (ids == null) { + eventHub.call(Events.CACHE, action, type); + } else { + if (ids.size() == 1) { + eventHub.call(Events.CACHE, action, type, ids.get(0)); + } else { + eventHub.call(Events.CACHE, action, type, ids.toArray()); + } + } } catch (RejectedExecutionException e) { LOG.warn("Can't update cache due to EventHub is too busy"); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index f8ac2fe47a..96e8833c0c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -64,12 +64,15 @@ public void save(SnapshotWriter writer, Closure done, // Write snapshot to real directory Set snapshotDirs = this.doSnapshotSave(); executor.execute(() -> { + long begin = System.currentTimeMillis(); Set tarSnapshotFiles = this.compressSnapshotDir(snapshotDirs, done); String jraftSnapshotPath = this.writeManifest(writer, tarSnapshotFiles, done); this.deleteSnapshotDir(snapshotDirs, done); this.compressJraftSnapshotDir(writer, jraftSnapshotPath, done); + LOG.info("Compress snapshot cost {}ms", + System.currentTimeMillis() - begin); }); } catch (Throwable t) { LOG.error("Failed to save snapshot", t); @@ -111,7 +114,7 @@ public boolean load(SnapshotReader reader) { private Set doSnapshotSave() { Set snapshotDirs = new HashSet<>(); for (RaftBackendStore store : this.stores) { - snapshotDirs.addAll(store.originStore().writeSnapshot(SNAPSHOT_DIR)); + snapshotDirs.addAll(store.originStore().createSnapshot(SNAPSHOT_DIR)); } LOG.info("All snapshot dirs: {}", snapshotDirs); return snapshotDirs; @@ -119,7 +122,7 @@ private Set doSnapshotSave() { private void doSnapshotLoad() { for (RaftBackendStore store : this.stores) { - store.originStore().readSnapshot(SNAPSHOT_DIR); + store.originStore().resumeSnapshot(SNAPSHOT_DIR); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java index 12efc35682..2b8981015c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreStateMachine.java @@ -40,7 +40,6 @@ import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.BytesBuffer; import com.baidu.hugegraph.backend.store.BackendAction; -import com.baidu.hugegraph.backend.store.BackendEntry; import com.baidu.hugegraph.backend.store.BackendMutation; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.raft.RaftBackendStore.IncrCounter; @@ -86,21 +85,13 @@ private void updateCacheIfNeeded(BackendMutation mutation, return; } for (HugeType type : mutation.types()) { - if (type.isSchema()) { - java.util.Iterator it = mutation.mutation(type); - while (it.hasNext()) { - BackendEntry entry = it.next().entry(); - this.context.notifyCache(Cache.ACTION_INVALID, type, - entry.originId()); - } - } else if (type.isGraph()) { - List ids = new ArrayList<>((int) Query.COMMIT_BATCH); + List ids = new ArrayList<>((int) Query.COMMIT_BATCH); + if (type.isSchema() || type.isGraph()) { java.util.Iterator it = mutation.mutation(type); while (it.hasNext()) { ids.add(it.next().entry().originId()); } - this.context.notifyCache(Cache.ACTION_INVALID, type, - ids.toArray()); + this.context.notifyCache(Cache.ACTION_INVALID, type, ids); } else { // Ignore other types due to not cached } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java index a97f385bea..7639c21003 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java @@ -103,6 +103,10 @@ public class HugeSecurityManager extends SecurityManager { "com.baidu.hugegraph.backend.store.hbase.HbaseSessions$RowIterator" ); + private static final Set ROCKSDB_CLASSES = ImmutableSet.of( + "com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore" + ); + private static final Set RAFT_CLASSES = ImmutableSet.of( "com.baidu.hugegraph.backend.store.raft.RaftNode", "com.baidu.hugegraph.backend.store.raft.StoreStateMachine", @@ -110,7 +114,8 @@ public class HugeSecurityManager extends SecurityManager { ); private static final Set SOFA_RPC_CLASSES = ImmutableSet.of( - "com.alipay.sofa.rpc.tracer.sofatracer.RpcSofaTracer" + "com.alipay.sofa.rpc.tracer.sofatracer.RpcSofaTracer", + "com.alipay.sofa.rpc.client.AbstractCluster" ); @Override @@ -204,7 +209,8 @@ public void checkRead(FileDescriptor fd) { public void checkRead(String file) { if (callFromGremlin() && !callFromCaffeine() && !readGroovyInCurrentDir(file) && !callFromBackendHbase() && - !callFromRaft() && !callFromSofaRpc()) { + !callFromBackendRocksDB() && !callFromRaft() && + !callFromSofaRpc()) { throw newSecurityException( "Not allowed to read file via Gremlin: %s", file); } @@ -231,7 +237,8 @@ public void checkWrite(FileDescriptor fd) { @Override public void checkWrite(String file) { - if (callFromGremlin() && !callFromRaft() && !callFromSofaRpc()) { + if (callFromGremlin() && !callFromBackendRocksDB() && + !callFromRaft() && !callFromSofaRpc()) { throw newSecurityException("Not allowed to write file via Gremlin"); } super.checkWrite(file); @@ -239,7 +246,7 @@ public void checkWrite(String file) { @Override public void checkDelete(String file) { - if (callFromGremlin()) { + if (callFromGremlin() && !callFromBackendRocksDB()) { throw newSecurityException( "Not allowed to delete file via Gremlin"); } @@ -322,7 +329,8 @@ public void checkPropertiesAccess() { public void checkPropertyAccess(String key) { if (!callFromAcceptClassLoaders() && callFromGremlin() && !WHITE_SYSTEM_PROPERTYS.contains(key) && !callFromBackendHbase() && - !callFromRaft() && !callFromSofaRpc()) { + !callFromBackendRocksDB() && !callFromRaft() && + !callFromSofaRpc()) { throw newSecurityException( "Not allowed to access system property(%s) via Gremlin", key); } @@ -442,6 +450,10 @@ private static boolean callFromBackendHbase() { return callFromWorkerWithClass(HBASE_CLASSES); } + private static boolean callFromBackendRocksDB() { + return callFromWorkerWithClass(ROCKSDB_CLASSES); + } + private static boolean callFromRaft() { return callFromWorkerWithClass(RAFT_CLASSES); } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index c3ed2528c7..f73899bdf9 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -608,7 +608,7 @@ protected Session session(HugeType tableType) { } @Override - public Set writeSnapshot(String snapshotPrefix) { + public Set createSnapshot(String snapshotPrefix) { Lock readLock = this.storeLock.readLock(); readLock.lock(); try { @@ -641,7 +641,7 @@ public Set writeSnapshot(String snapshotPrefix) { } @Override - public Set readSnapshot(String snapshotPrefix) { + public Set resumeSnapshot(String snapshotPrefix) { Lock readLock = this.storeLock.readLock(); readLock.lock(); try { diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index 7c070da341..af6609c8db 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -5489,7 +5489,11 @@ public void testRemoveEdgesOfSuperVertex() { guido.remove(); // Clear all - graph.truncateBackend(); + try { + graph.truncateBackend(); + } catch (UnsupportedOperationException ignored) { + // pass + } } @Test From 74d2a33cb1270a6a7dec33e724b0573bf14d5899 Mon Sep 17 00:00:00 2001 From: liningrui Date: Wed, 24 Mar 2021 15:42:25 +0800 Subject: [PATCH 06/13] Use checkpoint when resume snapshot Change-Id: Iebf742426aa6b696b2397ec90368a13d04c04b56 --- hugegraph-api/pom.xml | 2 +- .../hugegraph/api/profile/GraphsAPI.java | 4 +- .../baidu/hugegraph/version/ApiVersion.java | 3 +- .../baidu/hugegraph/StandardHugeGraph.java | 1 + .../store/AbstractBackendStoreProvider.java | 12 ++-- .../backend/store/BackendFeatures.java | 4 ++ .../hugegraph/backend/store/BackendStore.java | 4 +- .../store/raft/RaftBackendStoreProvider.java | 4 +- .../backend/store/raft/StoreSnapshotFile.java | 27 ++------ .../store/rocksdb/RocksDBFeatures.java | 5 ++ .../store/rocksdb/RocksDBSessions.java | 4 ++ .../store/rocksdb/RocksDBStdSessions.java | 50 ++++++++------- .../backend/store/rocksdb/RocksDBStore.java | 62 ++++++++++++------- .../store/rocksdbsst/RocksDBSstSessions.java | 8 ++- .../baidu/hugegraph/core/EdgeCoreTest.java | 6 +- 15 files changed, 110 insertions(+), 86 deletions(-) diff --git a/hugegraph-api/pom.xml b/hugegraph-api/pom.xml index b687b64ac2..ee70e207aa 100644 --- a/hugegraph-api/pom.xml +++ b/hugegraph-api/pom.xml @@ -171,7 +171,7 @@ - 0.59.0.0 + 0.60.0.0 diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java index 5003001ca8..eccb92c45c 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java @@ -140,7 +140,7 @@ public void clear(@Context GraphManager manager, @PUT @Timed - @Path("{name}/create-snapshot") + @Path("{name}/snapshot_create") @Produces(APPLICATION_JSON_WITH_CHARSET) @RolesAllowed({"admin", "$owner=$name"}) public Object createSnapshot(@Context GraphManager manager, @@ -154,7 +154,7 @@ public Object createSnapshot(@Context GraphManager manager, @PUT @Timed - @Path("{name}/resume-snapshot") + @Path("{name}/snapshot_resume") @Produces(APPLICATION_JSON_WITH_CHARSET) @RolesAllowed({"admin", "$owner=$name"}) public Object resumeSnapshot(@Context GraphManager manager, diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java index 4062ef3534..8b8babce64 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/version/ApiVersion.java @@ -109,10 +109,11 @@ public final class ApiVersion { * [0.58] Issue-1173: Supports customized kout/kneighbor, * multi-node-shortest-path, jaccard-similar and template-paths * [0.59] Issue-1333: Support graph read mode for olap property + * [0.60] Issue-1392: Support create and resume snapshot */ // The second parameter of Version.of() is for IDE running without JAR - public static final Version VERSION = Version.of(ApiVersion.class, "0.59"); + public static final Version VERSION = Version.of(ApiVersion.class, "0.60"); public static final void check() { // Check version of hugegraph-core. Firstly do check from version 0.3 diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index a627b73b79..3a16deb59e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -368,6 +368,7 @@ public void truncateBackend() { LOG.info("Graph '{}' has been truncated", this.name); } + @Override public void createSnapshot() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index 48aeed09b8..1099d41bb8 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -158,14 +158,12 @@ public void initSystemInfo(HugeGraph graph) { @Override public void createSnapshot() { - long begin = System.currentTimeMillis(); String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; for (BackendStore store : this.stores.values()) { - store.createSnapshot(snapshotPrefix); + if (store.features().supportsSnapshot()) { + store.createSnapshot(snapshotPrefix); + } } - - LOG.info("Create snapshot cost {}ms", - System.currentTimeMillis() - begin); } @Override @@ -173,7 +171,9 @@ public void resumeSnapshot() { String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; Set snapshotDirs = new HashSet<>(); for (BackendStore store : this.stores.values()) { - snapshotDirs.addAll(store.resumeSnapshot(snapshotPrefix)); + if (store.features().supportsSnapshot()) { + snapshotDirs.addAll(store.resumeSnapshot(snapshotPrefix)); + } } // Delete all snapshot parent directories for (String snapshotDir : snapshotDirs) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java index 49b83a3e36..e67c6d7beb 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java @@ -29,6 +29,10 @@ public default boolean supportsSharedStorage() { return true; } + public default boolean supportsSnapshot() { + return false; + } + public boolean supportsScanToken(); public boolean supportsScanKeyPrefix(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java index 42257fcdbb..f92d8a5b51 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java @@ -118,11 +118,11 @@ public default void setCounterLowest(HugeType type, long lowest) { public long getCounter(HugeType type); public default Set createSnapshot(String snapshotDir) { - throw new UnsupportedOperationException("writeSnapshot"); + throw new UnsupportedOperationException("createSnapshot"); } public default Set resumeSnapshot(String snapshotDir) { - throw new UnsupportedOperationException("readSnapshot"); + throw new UnsupportedOperationException("resumeSnapshot"); } static enum TxState { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java index 2f55018563..cabbd14d7f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java @@ -209,8 +209,8 @@ public void createSnapshot() { @Override public void resumeSnapshot() { - // jraft doesn't expose API to load snapshot - throw new UnsupportedOperationException(); + // Jraft doesn't expose API to load snapshot + throw new UnsupportedOperationException("resumeSnapshot"); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index 96e8833c0c..52132943be 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.nio.file.Paths; import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.zip.Checksum; @@ -65,11 +64,8 @@ public void save(SnapshotWriter writer, Closure done, Set snapshotDirs = this.doSnapshotSave(); executor.execute(() -> { long begin = System.currentTimeMillis(); - Set tarSnapshotFiles = - this.compressSnapshotDir(snapshotDirs, done); String jraftSnapshotPath = - this.writeManifest(writer, tarSnapshotFiles, done); - this.deleteSnapshotDir(snapshotDirs, done); + this.writeManifest(writer, snapshotDirs, done); this.compressJraftSnapshotDir(writer, jraftSnapshotPath, done); LOG.info("Compress snapshot cost {}ms", System.currentTimeMillis() - begin); @@ -93,7 +89,7 @@ public boolean load(SnapshotReader reader) { .toString(); try { // decompress manifest and data directory - this.decompressSnapshot(readerPath, jraftSnapshotPath, meta); + this.decompressSnapshot(readerPath, meta); this.doSnapshotLoad(); File tmp = new File(jraftSnapshotPath); // Delete the decompressed temporary file. If the deletion fails @@ -159,7 +155,7 @@ private void deleteSnapshotDir(Set snapshotDirs, } private String writeManifest(SnapshotWriter writer, - Set tarSnapshotFiles, + Set snapshotFiles, Closure done) { String writerPath = writer.getPath(); // Write all backend compressed snapshot file path to manifest @@ -167,7 +163,7 @@ private String writeManifest(SnapshotWriter writer, .toString(); File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); try { - FileUtils.writeLines(snapshotManifest, tarSnapshotFiles); + FileUtils.writeLines(snapshotManifest, snapshotFiles); } catch (IOException e) { done.run(new Status(RaftError.EIO, "Failed to write backend snapshot file path " + @@ -202,9 +198,8 @@ private void compressJraftSnapshotDir(SnapshotWriter writer, } } - private void decompressSnapshot(String readerPath, - String jraftSnapshotPath, - LocalFileMeta meta) throws IOException { + private void decompressSnapshot(String readerPath, LocalFileMeta meta) + throws IOException { String archiveFile = Paths.get(readerPath, SNAPSHOT_ARCHIVE).toString(); Checksum checksum = new CRC64(); CompressUtil.decompressTar(archiveFile, readerPath, checksum); @@ -213,15 +208,5 @@ private void decompressSnapshot(String readerPath, Long.toHexString(checksum.getValue())), "Snapshot checksum failed"); } - - File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); - List compressedSnapshotFiles = FileUtils.readLines( - snapshotManifest); - for (String compressedSnapshotFile : compressedSnapshotFiles) { - String targetDir = Paths.get(compressedSnapshotFile).getParent() - .toString(); - CompressUtil.decompressTar(compressedSnapshotFile, targetDir, - new CRC64()); - } } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java index cdbb00a8aa..a133845a9d 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java @@ -28,6 +28,11 @@ public boolean supportsSharedStorage() { return false; } + @Override + public boolean supportsSnapshot() { + return true; + } + @Override public boolean supportsScanToken() { return false; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java index 1889344ed5..4e81cc4d30 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.commons.lang3.tuple.Pair; +import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIterator; @@ -47,6 +48,9 @@ public RocksDBSessions(HugeConfig config, String database, String store) { public abstract RocksDBSessions copy(HugeConfig config, String database, String store); + public abstract RocksDB createSnapshotRocksDB(String snapshotPath) + throws RocksDBException; + public abstract void createSnapshot(String parentPath); public abstract void reload() throws RocksDBException; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 48191965c8..dc976bf668 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -20,8 +20,6 @@ package com.baidu.hugegraph.backend.store.rocksdb; import java.io.Closeable; -import java.io.File; -import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; @@ -35,11 +33,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.BloomFilter; -import org.rocksdb.Checkpoint; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ColumnFamilyOptions; @@ -324,27 +320,33 @@ public RocksDBSessions copy(HugeConfig config, } @Override - public void createSnapshot(String snapshotPath) { - // https://github.com/facebook/rocksdb/wiki/Checkpoints - try (Checkpoint checkpoint = Checkpoint.create(this.rocksdb)) { - String tempPath = snapshotPath + "_temp"; - File tempFile = new File(tempPath); - FileUtils.deleteDirectory(tempFile); - LOG.debug("Deleted temp directory {}", tempFile); - - FileUtils.forceMkdir(tempFile.getParentFile()); - checkpoint.createCheckpoint(tempPath); - File snapshotFile = new File(snapshotPath); - FileUtils.deleteDirectory(snapshotFile); - LOG.debug("Deleted stale directory {}", snapshotFile); - if (!tempFile.renameTo(snapshotFile)) { - throw new IOException(String.format("Failed to rename %s to %s", - tempFile, snapshotFile)); - } - } catch (Exception e) { - throw new BackendException("Failed to write snapshot at path %s", - e, snapshotPath); + public RocksDB createSnapshotRocksDB(String snapshotPath) + throws RocksDBException { + // Init CFs options + Set mergedCFs = this.mergeOldCFs(snapshotPath, new ArrayList<>( + this.cfs.keySet())); + List cfNames = ImmutableList.copyOf(mergedCFs); + + List cfds = new ArrayList<>(cfNames.size()); + for (String cf : cfNames) { + ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); + ColumnFamilyOptions options = cfd.getOptions(); + RocksDBStdSessions.initOptions(this.config, null, null, + options, options); + cfds.add(cfd); } + List cfhs = new ArrayList<>(); + + // Init DB options + DBOptions options = new DBOptions(); + RocksDBStdSessions.initOptions(this.config, options, options, + null, null); + return RocksDB.open(options, snapshotPath, cfds, cfhs); + } + + @Override + public void createSnapshot(String snapshotPath) { + RocksDBStore.createCheckpoint(this.rocksdb, snapshotPath); } @Override diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index f73899bdf9..0d7112e766 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -44,6 +44,8 @@ import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; +import org.rocksdb.Checkpoint; +import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.slf4j.Logger; @@ -65,7 +67,6 @@ import com.baidu.hugegraph.util.Consumers; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.ExecutorUtil; -import com.baidu.hugegraph.util.GZipUtil; import com.baidu.hugegraph.util.InsertionOrderUtil; import com.baidu.hugegraph.util.Log; import com.google.common.collect.ImmutableList; @@ -664,11 +665,19 @@ public Set resumeSnapshot(String snapshotPrefix) { Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + pureDataPath); E.checkState(snapshotPath.toFile().exists(), - "The snapshot path '%s' doesn't exist"); + "The snapshot path '%s' doesn't exist", + snapshotPath); LOG.debug("The origin data path: {}", originDataPath); LOG.debug("The snapshot data path: {}", snapshotPath); - fileNameMaps.put(originDataPath, snapshotPath); + RocksDBSessions sessions = entry.getValue(); + RocksDB rocksdb = sessions.createSnapshotRocksDB( + snapshotPath.toString()); + Path snapshotLinkPath = Paths.get(originDataPath + "_link"); + createCheckpoint(rocksdb, snapshotLinkPath.toString()); + rocksdb.close(); + + fileNameMaps.put(originDataPath, snapshotLinkPath); uniqueParents.add(snapshotPath.getParent().toString()); } E.checkState(!fileNameMaps.isEmpty(), @@ -684,19 +693,19 @@ public Set resumeSnapshot(String snapshotPrefix) { // Move snapshot file to origin data path for (Map.Entry entry : fileNameMaps.entrySet()) { File originDataDir = entry.getKey().toFile(); - File snapshotDir = entry.getValue().toFile(); + File snapshotLinkDir = entry.getValue().toFile(); try { if (originDataDir.exists()) { LOG.info("Delete origin data directory {}", originDataDir); FileUtils.deleteDirectory(originDataDir); } - FileUtils.moveDirectory(snapshotDir, originDataDir); + FileUtils.moveDirectory(snapshotLinkDir, originDataDir); LOG.info("Move snapshot directory {} to {}", - snapshotDir, originDataDir); - } catch (IOException e) { - throw new BackendException("Failed to move %s to %s", - e, snapshotDir, originDataDir); + snapshotLinkDir, originDataDir); + } catch (Exception e) { + throw new BackendException("Failed to move %s to %s", e, + snapshotLinkDir, originDataDir); } } // Reload rocksdb instance @@ -731,18 +740,6 @@ private final void closeSessions() { } } - private Session findMatchedSession(File snapshotFile) { - String fileName = snapshotFile.getName(); - for (Session session : this.session()) { - String md5 = GZipUtil.md5(session.dataPath()); - if (fileName.equals(md5)) { - return session; - } - } - throw new BackendException("Can't find matched session for " + - "snapshot file %s", snapshotFile); - } - private final List session() { this.checkOpened(); @@ -828,6 +825,29 @@ private static boolean existsOtherKeyspace(String dataPath) { return false; } + public static void createCheckpoint(RocksDB rocksdb, String targetPath) { + // https://github.com/facebook/rocksdb/wiki/Checkpoints + try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { + String tempPath = targetPath + "_temp"; + File tempFile = new File(tempPath); + FileUtils.deleteDirectory(tempFile); + LOG.debug("Deleted temp directory {}", tempFile); + + FileUtils.forceMkdir(tempFile.getParentFile()); + checkpoint.createCheckpoint(tempPath); + File snapshotFile = new File(targetPath); + FileUtils.deleteDirectory(snapshotFile); + LOG.debug("Deleted stale directory {}", snapshotFile); + if (!tempFile.renameTo(snapshotFile)) { + throw new IOException(String.format("Failed to rename %s to %s", + tempFile, snapshotFile)); + } + } catch (Exception e) { + throw new BackendException("Failed to create checkpoint at path %s", + e, targetPath); + } + } + /***************************** Store defines *****************************/ public static class RocksDBSchemaStore extends RocksDBStore { diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java index 1e5cce927e..f2a4db506f 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.EnvOptions; import org.rocksdb.Options; +import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.SstFileWriter; @@ -134,7 +135,7 @@ public boolean existsTable(String table) { @Override public List property(String property) { - throw new NotSupportException("RocksDBSstStore property()"); + throw new UnsupportedOperationException("RocksDBSstStore property()"); } @Override @@ -143,6 +144,11 @@ public RocksDBSessions copy(HugeConfig config, return new RocksDBSstSessions(config, database, store, this); } + @Override + public RocksDB createSnapshotRocksDB(String snapshotPath) { + throw new UnsupportedOperationException("createSnapshotRocksDB"); + } + @Override public void createSnapshot(String snapshotPath) { throw new UnsupportedOperationException("createSnapshot"); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index af6609c8db..7c070da341 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -5489,11 +5489,7 @@ public void testRemoveEdgesOfSuperVertex() { guido.remove(); // Clear all - try { - graph.truncateBackend(); - } catch (UnsupportedOperationException ignored) { - // pass - } + graph.truncateBackend(); } @Test From 084bca2de57ec2bec936dd98d44fb4d88872b282 Mon Sep 17 00:00:00 2001 From: liningrui Date: Thu, 25 Mar 2021 11:30:53 +0800 Subject: [PATCH 07/13] some improve Change-Id: Idd420fd98be6668cf0db03178530eda3b9529ea5 --- .../store/AbstractBackendStoreProvider.java | 22 +-- .../hugegraph/backend/store/BackendStore.java | 3 +- .../backend/store/raft/StoreSnapshotFile.java | 10 +- .../store/rocksdb/RocksDBSessions.java | 14 +- .../store/rocksdb/RocksDBStdSessions.java | 134 +++++++++++++++--- .../backend/store/rocksdb/RocksDBStore.java | 110 +++----------- .../store/rocksdbsst/RocksDBSstSessions.java | 17 ++- .../baidu/hugegraph/api/GremlinApiTest.java | 3 +- .../baidu/hugegraph/core/EdgeCoreTest.java | 6 +- 9 files changed, 168 insertions(+), 151 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index 1099d41bb8..fc35b7fb84 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -19,15 +19,10 @@ package com.baidu.hugegraph.backend.store; -import java.io.File; -import java.io.IOException; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; -import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; @@ -160,28 +155,15 @@ public void initSystemInfo(HugeGraph graph) { public void createSnapshot() { String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; for (BackendStore store : this.stores.values()) { - if (store.features().supportsSnapshot()) { - store.createSnapshot(snapshotPrefix); - } + store.createSnapshot(snapshotPrefix); } } @Override public void resumeSnapshot() { String snapshotPrefix = StoreSnapshotFile.SNAPSHOT_DIR; - Set snapshotDirs = new HashSet<>(); for (BackendStore store : this.stores.values()) { - if (store.features().supportsSnapshot()) { - snapshotDirs.addAll(store.resumeSnapshot(snapshotPrefix)); - } - } - // Delete all snapshot parent directories - for (String snapshotDir : snapshotDirs) { - try { - FileUtils.deleteDirectory(new File(snapshotDir)); - } catch (IOException e) { - LOG.warn("Failed to delete snapshot directory {}", snapshotDir); - } + store.resumeSnapshot(snapshotPrefix, true); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java index f92d8a5b51..7c55da1a3f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStore.java @@ -121,7 +121,8 @@ public default Set createSnapshot(String snapshotDir) { throw new UnsupportedOperationException("createSnapshot"); } - public default Set resumeSnapshot(String snapshotDir) { + public default void resumeSnapshot(String snapshotDir, + boolean deleteSnapshot) { throw new UnsupportedOperationException("resumeSnapshot"); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index 52132943be..02e89ad0e1 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -63,12 +63,10 @@ public void save(SnapshotWriter writer, Closure done, // Write snapshot to real directory Set snapshotDirs = this.doSnapshotSave(); executor.execute(() -> { - long begin = System.currentTimeMillis(); - String jraftSnapshotPath = - this.writeManifest(writer, snapshotDirs, done); + String jraftSnapshotPath = this.writeManifest(writer, + snapshotDirs, + done); this.compressJraftSnapshotDir(writer, jraftSnapshotPath, done); - LOG.info("Compress snapshot cost {}ms", - System.currentTimeMillis() - begin); }); } catch (Throwable t) { LOG.error("Failed to save snapshot", t); @@ -118,7 +116,7 @@ private Set doSnapshotSave() { private void doSnapshotLoad() { for (RaftBackendStore store : this.stores) { - store.originStore().resumeSnapshot(SNAPSHOT_DIR); + store.originStore().resumeSnapshot(SNAPSHOT_DIR, false); } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java index 4e81cc4d30..7172635e56 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java @@ -19,11 +19,11 @@ package com.baidu.hugegraph.backend.store.rocksdb; +import java.nio.file.Path; import java.util.List; import java.util.Set; import org.apache.commons.lang3.tuple.Pair; -import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIterator; @@ -48,12 +48,16 @@ public RocksDBSessions(HugeConfig config, String database, String store) { public abstract RocksDBSessions copy(HugeConfig config, String database, String store); - public abstract RocksDB createSnapshotRocksDB(String snapshotPath) - throws RocksDBException; - public abstract void createSnapshot(String parentPath); - public abstract void reload() throws RocksDBException; + public abstract void resumeSnapshot(String snapshotPath); + + public abstract Path buildSnapshotPath(Path originDataPath, + String snapshotPrefix, + boolean deleteSnapshot) + throws RocksDBException; + + public abstract void reloadRocksDB() throws RocksDBException; public abstract void forceCloseRocksDB(); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index dc976bf668..03b14db0cd 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -20,6 +20,8 @@ package com.baidu.hugegraph.backend.store.rocksdb; import java.io.Closeable; +import java.io.File; +import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; @@ -33,9 +35,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.BloomFilter; +import org.rocksdb.Checkpoint; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ColumnFamilyOptions; @@ -252,7 +256,7 @@ public synchronized void dropTable(String... tables) } @Override - public void reload() throws RocksDBException { + public void reloadRocksDB() throws RocksDBException { if (this.rocksdb.isOwningHandle()) { this.rocksdb.close(); } @@ -320,33 +324,72 @@ public RocksDBSessions copy(HugeConfig config, } @Override - public RocksDB createSnapshotRocksDB(String snapshotPath) - throws RocksDBException { - // Init CFs options - Set mergedCFs = this.mergeOldCFs(snapshotPath, new ArrayList<>( - this.cfs.keySet())); - List cfNames = ImmutableList.copyOf(mergedCFs); - - List cfds = new ArrayList<>(cfNames.size()); - for (String cf : cfNames) { - ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); - ColumnFamilyOptions options = cfd.getOptions(); - RocksDBStdSessions.initOptions(this.config, null, null, - options, options); - cfds.add(cfd); + public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, + boolean deleteSnapshot) + throws RocksDBException { + // originDataPath + // Like: parent_path/rocksdb-data/m + // parent_path/rocksdb-vertex/g + Path parentParentPath = originDataPath.getParent().getParent(); + // Like: rocksdb-data/m + // rocksdb-vertex/g + Path pureDataPath = parentParentPath.relativize(originDataPath); + // Like: parent_path/snapshot_rocksdb-data/m + // parent_path/snapshot_rocksdb-vertex/g + Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + + pureDataPath); + E.checkState(snapshotPath.toFile().exists(), + "The snapshot path '%s' doesn't exist", + snapshotPath); + LOG.debug("The origin data path: {}", originDataPath); + if (deleteSnapshot) { + LOG.debug("The snapshot data path: {}", snapshotPath); + return snapshotPath; } - List cfhs = new ArrayList<>(); - // Init DB options - DBOptions options = new DBOptions(); - RocksDBStdSessions.initOptions(this.config, options, options, - null, null); - return RocksDB.open(options, snapshotPath, cfds, cfhs); + RocksDB rocksdb = this.createSnapshotRocksDB(snapshotPath.toString()); + Path snapshotLinkPath = Paths.get(originDataPath + "_link"); + try { + createCheckpoint(rocksdb, snapshotLinkPath.toString()); + } finally { + rocksdb.close(); + } + LOG.debug("The snapshot data link path: {}", snapshotLinkPath); + return snapshotLinkPath; } @Override public void createSnapshot(String snapshotPath) { - RocksDBStore.createCheckpoint(this.rocksdb, snapshotPath); + createCheckpoint(this.rocksdb, snapshotPath); + } + + @Override + public void resumeSnapshot(String snapshotPath) { + File originDataDir = new File(this.dataPath); + File snapshotDir = new File(snapshotPath); + try { + /* + * Close current instance first + * NOTE: must close rocksdb instance before deleting file directory, + * if close after copying the snapshot directory to origin position, + * it may produce dirty data. + */ + this.forceCloseRocksDB(); + // Delete origin data directory + if (originDataDir.exists()) { + LOG.info("Delete origin data directory {}", originDataDir); + FileUtils.deleteDirectory(originDataDir); + } + // Move snapshot directory to origin data directory + FileUtils.moveDirectory(snapshotDir, originDataDir); + LOG.info("Move snapshot directory {} to {}", + snapshotDir, originDataDir); + // Reload rocksdb instance + this.reloadRocksDB(); + } catch (Exception e) { + throw new BackendException("Failed to resume snapshot '%s' to' %s'", + e, snapshotDir, this.dataPath); + } } @Override @@ -421,6 +464,30 @@ private void ingestExternalFile() throws RocksDBException { } } + private RocksDB createSnapshotRocksDB(String snapshotPath) + throws RocksDBException { + // Init CFs options + Set mergedCFs = this.mergeOldCFs(snapshotPath, new ArrayList<>( + this.cfs.keySet())); + List cfNames = ImmutableList.copyOf(mergedCFs); + + List cfds = new ArrayList<>(cfNames.size()); + for (String cf : cfNames) { + ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); + ColumnFamilyOptions options = cfd.getOptions(); + RocksDBStdSessions.initOptions(this.config, null, null, + options, options); + cfds.add(cfd); + } + List cfhs = new ArrayList<>(); + + // Init DB options + DBOptions options = new DBOptions(); + RocksDBStdSessions.initOptions(this.config, options, options, + null, null); + return RocksDB.open(options, snapshotPath, cfds, cfhs); + } + public static Set listCFs(String path) throws RocksDBException { Set cfs = new HashSet<>(); @@ -626,6 +693,29 @@ public static final String decode(byte[] bytes) { return StringEncoding.decode(bytes); } + public static void createCheckpoint(RocksDB rocksdb, String targetPath) { + // https://github.com/facebook/rocksdb/wiki/Checkpoints + try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { + String tempPath = targetPath + "_temp"; + File tempFile = new File(tempPath); + FileUtils.deleteDirectory(tempFile); + LOG.debug("Deleted temp directory {}", tempFile); + + FileUtils.forceMkdir(tempFile.getParentFile()); + checkpoint.createCheckpoint(tempPath); + File snapshotFile = new File(targetPath); + FileUtils.deleteDirectory(snapshotFile); + LOG.debug("Deleted stale directory {}", snapshotFile); + if (!tempFile.renameTo(snapshotFile)) { + throw new IOException(String.format("Failed to rename %s to %s", + tempFile, snapshotFile)); + } + } catch (Exception e) { + throw new BackendException("Failed to create checkpoint at path %s", + e, targetPath); + } + } + private class CFHandle implements Closeable { private final ColumnFamilyHandle handle; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 0d7112e766..827687ffde 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -19,8 +19,8 @@ package com.baidu.hugegraph.backend.store.rocksdb; -import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; @@ -44,8 +44,6 @@ import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; -import org.rocksdb.Checkpoint; -import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.slf4j.Logger; @@ -634,7 +632,7 @@ public Set createSnapshot(String snapshotPrefix) { uniqueParents.add(snapshotPath.getParent().toString()); } - LOG.info("The store '{}' save snapshot successfully", this.store); + LOG.info("The store '{}' create snapshot successfully", this.store); return uniqueParents; } finally { readLock.unlock(); @@ -642,80 +640,37 @@ public Set createSnapshot(String snapshotPrefix) { } @Override - public Set resumeSnapshot(String snapshotPrefix) { + public void resumeSnapshot(String snapshotPrefix, boolean deleteSnapshot) { Lock readLock = this.storeLock.readLock(); readLock.lock(); try { - Set uniqueParents = new HashSet<>(); if (!this.opened()) { - return uniqueParents; + return; } - - Map fileNameMaps = new HashMap<>(); + Map snapshotPaths = new HashMap<>(); for (Map.Entry entry : this.dbs.entrySet()) { - // Like: parent_path/rocksdb-data/m - // parent_path/rocksdb-vertex/g Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); - Path parentParentPath = originDataPath.getParent().getParent(); - // Like: rocksdb-data/m - // rocksdb-vertex/g - Path pureDataPath = parentParentPath.relativize(originDataPath); - // Like: parent_path/snapshot_rocksdb-data/m - // parent_path/snapshot_rocksdb-vertex/g - Path snapshotPath = parentParentPath.resolve(snapshotPrefix + - "_" + pureDataPath); - E.checkState(snapshotPath.toFile().exists(), - "The snapshot path '%s' doesn't exist", - snapshotPath); - LOG.debug("The origin data path: {}", originDataPath); - LOG.debug("The snapshot data path: {}", snapshotPath); + RocksDBSessions sessions = entry.getValue(); + Path snapshotPath = sessions.buildSnapshotPath(originDataPath, + snapshotPrefix, + deleteSnapshot); + snapshotPaths.put(snapshotPath, sessions); + } + for (Map.Entry entry : + snapshotPaths.entrySet()) { + Path snapshotPath = entry.getKey(); RocksDBSessions sessions = entry.getValue(); - RocksDB rocksdb = sessions.createSnapshotRocksDB( - snapshotPath.toString()); - Path snapshotLinkPath = Paths.get(originDataPath + "_link"); - createCheckpoint(rocksdb, snapshotLinkPath.toString()); - rocksdb.close(); + sessions.resumeSnapshot(snapshotPath.toString()); - fileNameMaps.put(originDataPath, snapshotLinkPath); - uniqueParents.add(snapshotPath.getParent().toString()); - } - E.checkState(!fileNameMaps.isEmpty(), - "The file name maps shouldn't be empty"); - /* - * NOTE: must close rocksdb instance before deleting file directory, - * if close after copying the snapshot directory to origin position, - * it may produce dirty data. - */ - for (RocksDBSessions sessions : this.sessions()) { - sessions.forceCloseRocksDB(); - } - // Move snapshot file to origin data path - for (Map.Entry entry : fileNameMaps.entrySet()) { - File originDataDir = entry.getKey().toFile(); - File snapshotLinkDir = entry.getValue().toFile(); - try { - if (originDataDir.exists()) { - LOG.info("Delete origin data directory {}", - originDataDir); - FileUtils.deleteDirectory(originDataDir); - } - FileUtils.moveDirectory(snapshotLinkDir, originDataDir); - LOG.info("Move snapshot directory {} to {}", - snapshotLinkDir, originDataDir); - } catch (Exception e) { - throw new BackendException("Failed to move %s to %s", e, - snapshotLinkDir, originDataDir); + Path parentPath = snapshotPath.getParent(); + if (Files.list(parentPath).count() == 0) { + FileUtils.deleteDirectory(parentPath.toFile()); } } - // Reload rocksdb instance - for (RocksDBSessions sessions : this.sessions()) { - sessions.reload(); - } - LOG.info("The store '{}' load snapshot successfully", this.store); - return uniqueParents; - } catch (RocksDBException e) { - throw new BackendException("Failed to reload rocksdb", e); + LOG.info("The store '{}' resume snapshot successfully", this.store); + } catch (RocksDBException | IOException e) { + throw new BackendException("Failed to resume snapshot", e); } finally { readLock.unlock(); } @@ -825,29 +780,6 @@ private static boolean existsOtherKeyspace(String dataPath) { return false; } - public static void createCheckpoint(RocksDB rocksdb, String targetPath) { - // https://github.com/facebook/rocksdb/wiki/Checkpoints - try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { - String tempPath = targetPath + "_temp"; - File tempFile = new File(tempPath); - FileUtils.deleteDirectory(tempFile); - LOG.debug("Deleted temp directory {}", tempFile); - - FileUtils.forceMkdir(tempFile.getParentFile()); - checkpoint.createCheckpoint(tempPath); - File snapshotFile = new File(targetPath); - FileUtils.deleteDirectory(snapshotFile); - LOG.debug("Deleted stale directory {}", snapshotFile); - if (!tempFile.renameTo(snapshotFile)) { - throw new IOException(String.format("Failed to rename %s to %s", - tempFile, snapshotFile)); - } - } catch (Exception e) { - throw new BackendException("Failed to create checkpoint at path %s", - e, targetPath); - } - } - /***************************** Store defines *****************************/ public static class RocksDBSchemaStore extends RocksDBStore { diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java index f2a4db506f..4a5a97ae7d 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java @@ -33,7 +33,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.EnvOptions; import org.rocksdb.Options; -import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.SstFileWriter; @@ -145,17 +144,23 @@ public RocksDBSessions copy(HugeConfig config, } @Override - public RocksDB createSnapshotRocksDB(String snapshotPath) { - throw new UnsupportedOperationException("createSnapshotRocksDB"); + public void createSnapshot(String snapshotPath) { + throw new UnsupportedOperationException("createSnapshot"); } @Override - public void createSnapshot(String snapshotPath) { - throw new UnsupportedOperationException("createSnapshot"); + public void resumeSnapshot(String snapshotPath) { + throw new UnsupportedOperationException("resumeSnapshot"); + } + + @Override + public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, + boolean deleteSnapshot) { + throw new UnsupportedOperationException("hardLink"); } @Override - public void reload() throws RocksDBException { + public void reloadRocksDB() throws RocksDBException { throw new UnsupportedOperationException("reload"); } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java index c2b6486b60..ded50a1b6c 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java @@ -111,7 +111,8 @@ public void testClearAndInit() { @Test public void testTruncate() { String body = "{" - + "\"gremlin\":\"hugegraph.truncateBackend()\"," + + "\"gremlin\":\"try {graph.truncateBackend()} " + + "catch (UnsupportedOperationException e) {}\"," + "\"bindings\":{}," + "\"language\":\"gremlin-groovy\"," + "\"aliases\":{\"g\":\"__g_hugegraph\"}}"; diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index 7c070da341..adf12ce989 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -5489,7 +5489,11 @@ public void testRemoveEdgesOfSuperVertex() { guido.remove(); // Clear all - graph.truncateBackend(); + try { + graph.truncateBackend(); + } catch (UnsupportedOperationException e) { + LOG.warn("Failed to create snapshot", e); + } } @Test From 060f190591f2c87eceadac9778f6ff62d88d28d8 Mon Sep 17 00:00:00 2001 From: liningrui Date: Fri, 26 Mar 2021 11:34:24 +0800 Subject: [PATCH 08/13] tiny improve Change-Id: I29adb7df352bbbba3fbd416831b11b0968818cb0 --- .../store/rocksdb/RocksDBStdSessions.java | 50 +++++++++---------- .../backend/store/rocksdb/RocksDBStore.java | 9 ++-- .../baidu/hugegraph/core/EdgeCoreTest.java | 2 +- 3 files changed, 32 insertions(+), 29 deletions(-) diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 03b14db0cd..e4026bc296 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -350,7 +350,7 @@ public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, RocksDB rocksdb = this.createSnapshotRocksDB(snapshotPath.toString()); Path snapshotLinkPath = Paths.get(originDataPath + "_link"); try { - createCheckpoint(rocksdb, snapshotLinkPath.toString()); + this.createCheckpoint(rocksdb, snapshotLinkPath.toString()); } finally { rocksdb.close(); } @@ -360,7 +360,7 @@ public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, @Override public void createSnapshot(String snapshotPath) { - createCheckpoint(this.rocksdb, snapshotPath); + this.createCheckpoint(this.rocksdb, snapshotPath); } @Override @@ -488,6 +488,29 @@ private RocksDB createSnapshotRocksDB(String snapshotPath) return RocksDB.open(options, snapshotPath, cfds, cfhs); } + private void createCheckpoint(RocksDB rocksdb, String targetPath) { + // https://github.com/facebook/rocksdb/wiki/Checkpoints + try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { + String tempPath = targetPath + "_temp"; + File tempFile = new File(tempPath); + FileUtils.deleteDirectory(tempFile); + LOG.debug("Deleted temp directory {}", tempFile); + + FileUtils.forceMkdir(tempFile.getParentFile()); + checkpoint.createCheckpoint(tempPath); + File snapshotFile = new File(targetPath); + FileUtils.deleteDirectory(snapshotFile); + LOG.debug("Deleted stale directory {}", snapshotFile); + if (!tempFile.renameTo(snapshotFile)) { + throw new IOException(String.format("Failed to rename %s to %s", + tempFile, snapshotFile)); + } + } catch (Exception e) { + throw new BackendException("Failed to create checkpoint at path %s", + e, targetPath); + } + } + public static Set listCFs(String path) throws RocksDBException { Set cfs = new HashSet<>(); @@ -693,29 +716,6 @@ public static final String decode(byte[] bytes) { return StringEncoding.decode(bytes); } - public static void createCheckpoint(RocksDB rocksdb, String targetPath) { - // https://github.com/facebook/rocksdb/wiki/Checkpoints - try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { - String tempPath = targetPath + "_temp"; - File tempFile = new File(tempPath); - FileUtils.deleteDirectory(tempFile); - LOG.debug("Deleted temp directory {}", tempFile); - - FileUtils.forceMkdir(tempFile.getParentFile()); - checkpoint.createCheckpoint(tempPath); - File snapshotFile = new File(targetPath); - FileUtils.deleteDirectory(snapshotFile); - LOG.debug("Deleted stale directory {}", snapshotFile); - if (!tempFile.renameTo(snapshotFile)) { - throw new IOException(String.format("Failed to rename %s to %s", - tempFile, snapshotFile)); - } - } catch (Exception e) { - throw new BackendException("Failed to create checkpoint at path %s", - e, targetPath); - } - } - private class CFHandle implements Closeable { private final ColumnFamilyHandle handle; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 827687ffde..544baa911f 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -663,9 +663,12 @@ public void resumeSnapshot(String snapshotPrefix, boolean deleteSnapshot) { RocksDBSessions sessions = entry.getValue(); sessions.resumeSnapshot(snapshotPath.toString()); - Path parentPath = snapshotPath.getParent(); - if (Files.list(parentPath).count() == 0) { - FileUtils.deleteDirectory(parentPath.toFile()); + if (deleteSnapshot) { + // Delete empty snapshot parent directory + Path parentPath = snapshotPath.getParent(); + if (Files.list(parentPath).count() == 0) { + FileUtils.deleteDirectory(parentPath.toFile()); + } } } LOG.info("The store '{}' resume snapshot successfully", this.store); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index adf12ce989..b76ee9a763 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -5492,7 +5492,7 @@ public void testRemoveEdgesOfSuperVertex() { try { graph.truncateBackend(); } catch (UnsupportedOperationException e) { - LOG.warn("Failed to create snapshot", e); + LOG.warn("Failed to truncate backend", e); } } From fcb58603541139af7b1aebcc61705a5b7735537f Mon Sep 17 00:00:00 2001 From: liningrui Date: Fri, 26 Mar 2021 15:59:43 +0800 Subject: [PATCH 09/13] Refactor some code in RocksDBStdSessions Change-Id: I0561be1a71322644465e745ded9ca1636bff078a --- .../backend/store/raft/StoreSnapshotFile.java | 4 +- .../security/HugeSecurityManager.java | 11 +- .../store/rocksdb/RocksDBSessions.java | 10 +- .../store/rocksdb/RocksDBStdSessions.java | 345 +++++++++--------- .../backend/store/rocksdb/RocksDBStore.java | 29 +- .../store/rocksdbsst/RocksDBSstSessions.java | 13 +- .../baidu/hugegraph/api/GremlinApiTest.java | 2 +- 7 files changed, 209 insertions(+), 205 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index 02e89ad0e1..e3d8d77bd1 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -159,9 +159,9 @@ private String writeManifest(SnapshotWriter writer, // Write all backend compressed snapshot file path to manifest String jraftSnapshotPath = Paths.get(writerPath, SNAPSHOT_DIR) .toString(); - File snapshotManifest = new File(jraftSnapshotPath, MANIFEST); + File snapshotManifestFile = new File(jraftSnapshotPath, MANIFEST); try { - FileUtils.writeLines(snapshotManifest, snapshotFiles); + FileUtils.writeLines(snapshotManifestFile, snapshotFiles); } catch (IOException e) { done.run(new Status(RaftError.EIO, "Failed to write backend snapshot file path " + diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java index 7639c21003..8651c6c4c1 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java @@ -95,6 +95,11 @@ public class HugeSecurityManager extends SecurityManager { ImmutableSet.of("execute") ); + private static final Map> ROCKSDB_SNAPSHOT = ImmutableMap.of( + "com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore", + ImmutableSet.of("createSnapshot", "resumeSnapshot") + ); + private static final Set HBASE_CLASSES = ImmutableSet.of( // Fixed #758 "com.baidu.hugegraph.backend.store.hbase.HbaseStore", @@ -103,10 +108,6 @@ public class HugeSecurityManager extends SecurityManager { "com.baidu.hugegraph.backend.store.hbase.HbaseSessions$RowIterator" ); - private static final Set ROCKSDB_CLASSES = ImmutableSet.of( - "com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore" - ); - private static final Set RAFT_CLASSES = ImmutableSet.of( "com.baidu.hugegraph.backend.store.raft.RaftNode", "com.baidu.hugegraph.backend.store.raft.StoreStateMachine", @@ -451,7 +452,7 @@ private static boolean callFromBackendHbase() { } private static boolean callFromBackendRocksDB() { - return callFromWorkerWithClass(ROCKSDB_CLASSES); + return callFromMethods(ROCKSDB_SNAPSHOT); } private static boolean callFromRaft() { diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java index 7172635e56..e92f358a4d 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBSessions.java @@ -19,13 +19,13 @@ package com.baidu.hugegraph.backend.store.rocksdb; -import java.nio.file.Path; import java.util.List; import java.util.Set; import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.RocksDBException; +import com.alipay.sofa.jraft.storage.snapshot.remote.Session; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import com.baidu.hugegraph.backend.store.BackendSession.AbstractBackendSession; import com.baidu.hugegraph.backend.store.BackendSessionPool; @@ -52,10 +52,10 @@ public abstract RocksDBSessions copy(HugeConfig config, public abstract void resumeSnapshot(String snapshotPath); - public abstract Path buildSnapshotPath(Path originDataPath, - String snapshotPrefix, - boolean deleteSnapshot) - throws RocksDBException; + public abstract String buildSnapshotPath(String snapshotPrefix); + + public abstract String hardLinkSnapshot(String snapshotPath) + throws RocksDBException; public abstract void reloadRocksDB() throws RocksDBException; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index e4026bc296..06899d58db 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -82,10 +82,7 @@ public class RocksDBStdSessions extends RocksDBSessions { private final String dataPath; private final String walPath; - private volatile RocksDB rocksdb; - private final SstFileManager sstFileManager; - - private final Map cfs; + private volatile OpenedRocksDB rocksdb; private final AtomicInteger refCount; public RocksDBStdSessions(HugeConfig config, String database, String store, @@ -95,22 +92,8 @@ public RocksDBStdSessions(HugeConfig config, String database, String store, this.config = config; this.dataPath = dataPath; this.walPath = walPath; - // Init options - Options options = new Options(); - RocksDBStdSessions.initOptions(config, options, options, - options, options); - options.setWalDir(walPath); - - this.sstFileManager = new SstFileManager(Env.getDefault()); - options.setSstFileManager(this.sstFileManager); - - /* - * Open RocksDB at the first time - * Don't merge old CFs, we expect a clear DB when using this one - */ - this.rocksdb = RocksDB.open(options, dataPath); - - this.cfs = new ConcurrentHashMap<>(); + this.rocksdb = RocksDBStdSessions.openRocksDB(config, dataPath, + walPath); this.refCount = new AtomicInteger(1); } @@ -121,43 +104,11 @@ public RocksDBStdSessions(HugeConfig config, String database, String store, this.config = config; this.dataPath = dataPath; this.walPath = walPath; - // Old CFs should always be opened - Set mergedCFs = this.mergeOldCFs(dataPath, cfNames); - List cfs = ImmutableList.copyOf(mergedCFs); - - // Init CFs options - List cfds = new ArrayList<>(cfs.size()); - for (String cf : cfs) { - ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); - ColumnFamilyOptions options = cfd.getOptions(); - RocksDBStdSessions.initOptions(config, null, null, - options, options); - cfds.add(cfd); - } - - // Init DB options - DBOptions options = new DBOptions(); - RocksDBStdSessions.initOptions(config, options, options, null, null); - options.setWalDir(walPath); - - this.sstFileManager = new SstFileManager(Env.getDefault()); - options.setSstFileManager(this.sstFileManager); - - // Open RocksDB with CFs - List cfhs = new ArrayList<>(); - this.rocksdb = RocksDB.open(options, dataPath, cfds, cfhs); - E.checkState(cfhs.size() == cfs.size(), - "Expect same size of cf-handles and cf-names"); - - // Collect CF Handles - this.cfs = new ConcurrentHashMap<>(); - for (int i = 0; i < cfs.size(); i++) { - this.cfs.put(cfs.get(i), new CFHandle(cfhs.get(i))); - } - + this.rocksdb = RocksDBStdSessions.openRocksDB(config, cfNames, + dataPath, walPath); this.refCount = new AtomicInteger(1); - ingestExternalFile(); + this.ingestExternalFile(); } private RocksDBStdSessions(HugeConfig config, String database, String store, @@ -167,10 +118,7 @@ private RocksDBStdSessions(HugeConfig config, String database, String store, this.dataPath = origin.dataPath; this.walPath = origin.walPath; this.rocksdb = origin.rocksdb; - this.sstFileManager = origin.sstFileManager; - this.cfs = origin.cfs; this.refCount = origin.refCount; - this.refCount.incrementAndGet(); } @@ -181,12 +129,12 @@ public void open() throws Exception { @Override protected boolean opened() { - return this.rocksdb != null && this.rocksdb.isOwningHandle(); + return this.rocksdb != null && this.rocksdb.rocksdb.isOwningHandle(); } @Override public Set openedTables() { - return this.cfs.keySet(); + return this.rocksdb.cfs(); } @Override @@ -196,7 +144,7 @@ public synchronized void createTable(String... tables) List cfds = new ArrayList<>(); for (String table : tables) { - if (this.cfs.containsKey(table)) { + if (this.rocksdb.existCf(table)) { continue; } ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor( @@ -210,14 +158,14 @@ public synchronized void createTable(String... tables) * To speed up the creation of tables, like truncate() for tinkerpop * test, we call createColumnFamilies instead of createColumnFamily. */ - List cfhs = this.rocksdb.createColumnFamilies(cfds); + List cfhs = this.rocksdb().createColumnFamilies(cfds); for (ColumnFamilyHandle cfh : cfhs) { String table = decode(cfh.getName()); - this.cfs.put(table, new CFHandle(cfh)); + this.rocksdb.addCf(table, new CFHandle(cfh)); } - ingestExternalFile(); + this.ingestExternalFile(); } @Override @@ -232,7 +180,7 @@ public synchronized void dropTable(String... tables) */ List cfhs = new ArrayList<>(); for (String table : tables) { - CFHandle cfh = this.cfs.get(table); + CFHandle cfh = this.rocksdb.cf(table); if (cfh == null) { continue; } @@ -243,49 +191,32 @@ public synchronized void dropTable(String... tables) * To speed up the creation of tables, like truncate() for tinkerpop * test, we call dropColumnFamilies instead of dropColumnFamily. */ - this.rocksdb.dropColumnFamilies(cfhs); + this.rocksdb().dropColumnFamilies(cfhs); for (String table : tables) { - CFHandle cfh = this.cfs.get(table); + CFHandle cfh = this.rocksdb.cf(table); if (cfh == null) { continue; } cfh.destroy(); - this.cfs.remove(table); + this.rocksdb.removeCf(table); } } + @Override + public boolean existsTable(String table) { + return this.rocksdb.existCf(table); + } + @Override public void reloadRocksDB() throws RocksDBException { if (this.rocksdb.isOwningHandle()) { this.rocksdb.close(); } - this.cfs.values().forEach(CFHandle::destroy); - // Init CFs options - Set mergedCFs = this.mergeOldCFs(this.dataPath, new ArrayList<>( - this.cfs.keySet())); - List cfNames = ImmutableList.copyOf(mergedCFs); - - List cfds = new ArrayList<>(cfNames.size()); - for (String cf : cfNames) { - ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); - ColumnFamilyOptions options = cfd.getOptions(); - RocksDBStdSessions.initOptions(this.config, null, null, - options, options); - cfds.add(cfd); - } - List cfhs = new ArrayList<>(); - - // Init DB options - DBOptions options = new DBOptions(); - RocksDBStdSessions.initOptions(this.config, options, options, - null, null); - options.setWalDir(this.walPath); - options.setSstFileManager(this.sstFileManager); - this.rocksdb = RocksDB.open(options, this.dataPath, cfds, cfhs); - for (int i = 0; i < cfNames.size(); i++) { - this.cfs.put(cfNames.get(i), new CFHandle(cfhs.get(i))); - } + this.rocksdb = RocksDBStdSessions.openRocksDB(this.config, + ImmutableList.of(), + this.dataPath, + this.walPath); } @Override @@ -293,22 +224,17 @@ public void forceCloseRocksDB() { this.rocksdb().close(); } - @Override - public boolean existsTable(String table) { - return this.cfs.containsKey(table); - } - @Override public List property(String property) { try { if (property.equals(RocksDBMetrics.DISK_USAGE)) { - long size = this.sstFileManager.getTotalSize(); + long size = this.rocksdb.sstFileManager.getTotalSize(); return ImmutableList.of(String.valueOf(size)); } List values = new ArrayList<>(); for (String cf : this.openedTables()) { - try (CFHandle cfh = cf(cf)) { - values.add(rocksdb().getProperty(cfh.get(), property)); + try (CFHandle cfh = this.cf(cf)) { + values.add(this.rocksdb().getProperty(cfh.get(), property)); } } return values; @@ -323,44 +249,9 @@ public RocksDBSessions copy(HugeConfig config, return new RocksDBStdSessions(config, database, store, this); } - @Override - public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, - boolean deleteSnapshot) - throws RocksDBException { - // originDataPath - // Like: parent_path/rocksdb-data/m - // parent_path/rocksdb-vertex/g - Path parentParentPath = originDataPath.getParent().getParent(); - // Like: rocksdb-data/m - // rocksdb-vertex/g - Path pureDataPath = parentParentPath.relativize(originDataPath); - // Like: parent_path/snapshot_rocksdb-data/m - // parent_path/snapshot_rocksdb-vertex/g - Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + - pureDataPath); - E.checkState(snapshotPath.toFile().exists(), - "The snapshot path '%s' doesn't exist", - snapshotPath); - LOG.debug("The origin data path: {}", originDataPath); - if (deleteSnapshot) { - LOG.debug("The snapshot data path: {}", snapshotPath); - return snapshotPath; - } - - RocksDB rocksdb = this.createSnapshotRocksDB(snapshotPath.toString()); - Path snapshotLinkPath = Paths.get(originDataPath + "_link"); - try { - this.createCheckpoint(rocksdb, snapshotLinkPath.toString()); - } finally { - rocksdb.close(); - } - LOG.debug("The snapshot data link path: {}", snapshotLinkPath); - return snapshotLinkPath; - } - @Override public void createSnapshot(String snapshotPath) { - this.createCheckpoint(this.rocksdb, snapshotPath); + RocksDBStdSessions.createCheckpoint(this.rocksdb(), snapshotPath); } @Override @@ -392,6 +283,33 @@ public void resumeSnapshot(String snapshotPath) { } } + @Override + public String buildSnapshotPath(String snapshotPrefix) { + // Like: parent_path/rocksdb-data/*, * maybe g,m,s + Path originDataPath = Paths.get(this.dataPath); + Path parentParentPath = originDataPath.getParent().getParent(); + // Like: rocksdb-data/* + Path pureDataPath = parentParentPath.relativize(originDataPath); + // Like: parent_path/snapshot_rocksdb-data/* + Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + + pureDataPath); + E.checkState(snapshotPath.toFile().exists(), + "The snapshot path '%s' doesn't exist", + snapshotPath); + return snapshotPath.toString(); + } + + @Override + public String hardLinkSnapshot(String snapshotPath) throws RocksDBException { + String snapshotLinkPath = this.dataPath + "_link"; + try (RocksDB rocksdb = openRocksDB(this.config, ImmutableList.of(), + snapshotPath, null).rocksdb) { + RocksDBStdSessions.createCheckpoint(rocksdb, snapshotLinkPath); + } + LOG.debug("The snapshot data link path: {}", snapshotLinkPath); + return snapshotLinkPath; + } + @Override public final Session session() { return (Session) super.getOrNewSession(); @@ -412,12 +330,6 @@ protected synchronized void doClose() { return; } assert this.refCount.get() == 0; - - for (CFHandle cf : this.cfs.values()) { - cf.close(); - } - this.cfs.clear(); - this.rocksdb.close(); } @@ -428,67 +340,104 @@ private void checkValid() { private RocksDB rocksdb() { this.checkValid(); - return this.rocksdb; + return this.rocksdb.rocksdb; } - private CFHandle cf(String cf) { - CFHandle cfh = this.cfs.get(cf); + private CFHandle cf(String cfName) { + CFHandle cfh = this.rocksdb.cf(cfName); if (cfh == null) { - throw new BackendException("Table '%s' is not opened", cf); + throw new BackendException("Table '%s' is not opened", cfName); } cfh.open(); return cfh; } - private Set mergeOldCFs(String path, List cfNames) - throws RocksDBException { - Set cfs = listCFs(path); - cfs.addAll(cfNames); - return cfs; - } - private void ingestExternalFile() throws RocksDBException { String directory = this.config().get(RocksDBOptions.SST_PATH); if (directory == null || directory.isEmpty()) { return; } - RocksDBIngester ingester = new RocksDBIngester(this.rocksdb); + RocksDBIngester ingester = new RocksDBIngester(this.rocksdb()); // Ingest all *.sst files in `directory` - for (String cf : this.cfs.keySet()) { + for (String cf : this.rocksdb.cfs()) { Path path = Paths.get(directory, cf); if (path.toFile().isDirectory()) { - try (CFHandle cfh = cf(cf)) { + try (CFHandle cfh = this.cf(cf)) { ingester.ingest(path, cfh.get()); } } } } - private RocksDB createSnapshotRocksDB(String snapshotPath) - throws RocksDBException { - // Init CFs options - Set mergedCFs = this.mergeOldCFs(snapshotPath, new ArrayList<>( - this.cfs.keySet())); - List cfNames = ImmutableList.copyOf(mergedCFs); + private static OpenedRocksDB openRocksDB(HugeConfig config, + String dataPath, String walPath) + throws RocksDBException { + // Init options + Options options = new Options(); + RocksDBStdSessions.initOptions(config, options, options, + options, options); + options.setWalDir(walPath); + SstFileManager sstFileManager = new SstFileManager(Env.getDefault()); + options.setSstFileManager(sstFileManager); + /* + * Open RocksDB at the first time + * Don't merge old CFs, we expect a clear DB when using this one + */ + RocksDB rocksdb = RocksDB.open(options, dataPath); + Map cfs = new ConcurrentHashMap<>(); + return new OpenedRocksDB(rocksdb, cfs, sstFileManager); + } + + private static OpenedRocksDB openRocksDB(HugeConfig config, + List cfNames, + String dataPath, String walPath) + throws RocksDBException { + // Old CFs should always be opened + Set mergedCFs = RocksDBStdSessions.mergeOldCFs(dataPath, + cfNames); + List cfs = ImmutableList.copyOf(mergedCFs); - List cfds = new ArrayList<>(cfNames.size()); - for (String cf : cfNames) { + // Init CFs options + List cfds = new ArrayList<>(cfs.size()); + for (String cf : cfs) { ColumnFamilyDescriptor cfd = new ColumnFamilyDescriptor(encode(cf)); ColumnFamilyOptions options = cfd.getOptions(); - RocksDBStdSessions.initOptions(this.config, null, null, + RocksDBStdSessions.initOptions(config, null, null, options, options); cfds.add(cfd); } - List cfhs = new ArrayList<>(); // Init DB options DBOptions options = new DBOptions(); - RocksDBStdSessions.initOptions(this.config, options, options, - null, null); - return RocksDB.open(options, snapshotPath, cfds, cfhs); + RocksDBStdSessions.initOptions(config, options, options, null, null); + if (walPath != null) { + options.setWalDir(walPath); + } + SstFileManager sstFileManager = new SstFileManager(Env.getDefault()); + options.setSstFileManager(sstFileManager); + + // Open RocksDB with CFs + List cfhs = new ArrayList<>(); + RocksDB rocksdb = RocksDB.open(options, dataPath, cfds, cfhs); + + E.checkState(cfhs.size() == cfs.size(), + "Expect same size of cf-handles and cf-names"); + // Collect CF Handles + Map cfHandles = new ConcurrentHashMap<>(); + for (int i = 0; i < cfs.size(); i++) { + cfHandles.put(cfs.get(i), new CFHandle(cfhs.get(i))); + } + return new OpenedRocksDB(rocksdb, cfHandles, sstFileManager); + } + + private static Set mergeOldCFs(String path, List cfNames) + throws RocksDBException { + Set cfs = listCFs(path); + cfs.addAll(cfNames); + return cfs; } - private void createCheckpoint(RocksDB rocksdb, String targetPath) { + private static void createCheckpoint(RocksDB rocksdb, String targetPath) { // https://github.com/facebook/rocksdb/wiki/Checkpoints try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { String tempPath = targetPath + "_temp"; @@ -716,7 +665,57 @@ public static final String decode(byte[] bytes) { return StringEncoding.decode(bytes); } - private class CFHandle implements Closeable { + private static class OpenedRocksDB { + + private final RocksDB rocksdb; + private final Map cfHandles; + private final SstFileManager sstFileManager; + + public OpenedRocksDB(RocksDB rocksdb, Map cfHandles, + SstFileManager sstFileManager) { + this.rocksdb = rocksdb; + this.cfHandles = cfHandles; + this.sstFileManager = sstFileManager; + } + + public Set cfs() { + return this.cfHandles.keySet(); + } + + public CFHandle cf(String cfName) { + return this.cfHandles.get(cfName); + } + + public void addCf(String cfName, CFHandle cfHandle) { + this.cfHandles.put(cfName, cfHandle); + } + + public CFHandle removeCf(String cfName) { + return this.cfHandles.remove(cfName); + } + + public boolean existCf(String cfName) { + return this.cfHandles.containsKey(cfName); + } + + public boolean isOwningHandle() { + return this.rocksdb.isOwningHandle(); + } + + public void close() { + if (!this.isOwningHandle()) { + return; + } + for (CFHandle cf : this.cfHandles.values()) { + cf.close(); + } + this.cfHandles.clear(); + + this.rocksdb.close(); + } + } + + private static class CFHandle implements Closeable { private final ColumnFamilyHandle handle; private final AtomicInteger refs; diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 544baa911f..579ad2fa6e 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -614,15 +614,12 @@ public Set createSnapshot(String snapshotPrefix) { Set uniqueParents = new HashSet<>(); // Every rocksdb instance should create an snapshot for (Map.Entry entry : this.dbs.entrySet()) { - // Like: parent_path/rocksdb-data/m - // parent_path/rocksdb-vertex/g + // Like: parent_path/rocksdb-data/*, * maybe g,m,s Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); Path parentParentPath = originDataPath.getParent().getParent(); - // Like: rocksdb-data/m - // rocksdb-vertex/g + // Like: rocksdb-data/* Path pureDataPath = parentParentPath.relativize(originDataPath); - // Like: parent_path/snapshot_rocksdb-data/m - // parent_path/snapshot_rocksdb-vertex/g + // Like: parent_path/snapshot_rocksdb-data/* Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + pureDataPath); LOG.debug("The origin data path: {}", originDataPath); @@ -647,25 +644,27 @@ public void resumeSnapshot(String snapshotPrefix, boolean deleteSnapshot) { if (!this.opened()) { return; } - Map snapshotPaths = new HashMap<>(); + Map snapshotPaths = new HashMap<>(); for (Map.Entry entry : this.dbs.entrySet()) { - Path originDataPath = Paths.get(entry.getKey()).toAbsolutePath(); RocksDBSessions sessions = entry.getValue(); - Path snapshotPath = sessions.buildSnapshotPath(originDataPath, - snapshotPrefix, - deleteSnapshot); + String snapshotPath = sessions.buildSnapshotPath(snapshotPrefix); + LOG.debug("The origin data path: {}", entry.getKey()); + if (!deleteSnapshot) { + snapshotPath = sessions.hardLinkSnapshot(snapshotPath); + } + LOG.debug("The snapshot data path: {}", snapshotPath); snapshotPaths.put(snapshotPath, sessions); } - for (Map.Entry entry : + for (Map.Entry entry : snapshotPaths.entrySet()) { - Path snapshotPath = entry.getKey(); + String snapshotPath = entry.getKey(); RocksDBSessions sessions = entry.getValue(); - sessions.resumeSnapshot(snapshotPath.toString()); + sessions.resumeSnapshot(snapshotPath); if (deleteSnapshot) { // Delete empty snapshot parent directory - Path parentPath = snapshotPath.getParent(); + Path parentPath = Paths.get(snapshotPath).getParent(); if (Files.list(parentPath).count() == 0) { FileUtils.deleteDirectory(parentPath.toFile()); } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java index 4a5a97ae7d..b1b222cf9f 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstSessions.java @@ -154,14 +154,19 @@ public void resumeSnapshot(String snapshotPath) { } @Override - public Path buildSnapshotPath(Path originDataPath, String snapshotPrefix, - boolean deleteSnapshot) { - throw new UnsupportedOperationException("hardLink"); + public String buildSnapshotPath(String snapshotPrefix) { + throw new UnsupportedOperationException("buildSnapshotPath"); + } + + @Override + public String hardLinkSnapshot(String snapshotPath) + throws RocksDBException { + throw new UnsupportedOperationException("hardLinkSnapshot"); } @Override public void reloadRocksDB() throws RocksDBException { - throw new UnsupportedOperationException("reload"); + throw new UnsupportedOperationException("reloadRocksDB"); } @Override diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java index ded50a1b6c..19ad475add 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/GremlinApiTest.java @@ -111,7 +111,7 @@ public void testClearAndInit() { @Test public void testTruncate() { String body = "{" - + "\"gremlin\":\"try {graph.truncateBackend()} " + + "\"gremlin\":\"try {hugegraph.truncateBackend()} " + "catch (UnsupportedOperationException e) {}\"," + "\"bindings\":{}," + "\"language\":\"gremlin-groovy\"," From 9ffda82f209c45024654a9977fa20e7adea5ee8b Mon Sep 17 00:00:00 2001 From: liningrui Date: Fri, 26 Mar 2021 18:54:27 +0800 Subject: [PATCH 10/13] tiny improve Change-Id: I045690d244ad83e81e7ea7624ae9388f88521142 --- .../baidu/hugegraph/security/HugeSecurityManager.java | 4 +++- .../backend/store/rocksdb/RocksDBStdSessions.java | 9 ++++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java index 8651c6c4c1..dc870d800c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java @@ -96,7 +96,9 @@ public class HugeSecurityManager extends SecurityManager { ); private static final Map> ROCKSDB_SNAPSHOT = ImmutableMap.of( - "com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore", + "com.baidu.hugegraph.backend.store.AbstractBackendStoreProvider", + ImmutableSet.of("createSnapshot", "resumeSnapshot"), + "com.baidu.hugegraph.backend.store.raft.RaftBackendStoreProvider", ImmutableSet.of("createSnapshot", "resumeSnapshot") ); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 06899d58db..1224c65e78 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -285,7 +285,7 @@ public void resumeSnapshot(String snapshotPath) { @Override public String buildSnapshotPath(String snapshotPrefix) { - // Like: parent_path/rocksdb-data/*, * maybe g,m,s + // Like: parent_path/rocksdb-data/*, * can be g,m,s Path originDataPath = Paths.get(this.dataPath); Path parentParentPath = originDataPath.getParent().getParent(); // Like: rocksdb-data/* @@ -306,7 +306,8 @@ public String hardLinkSnapshot(String snapshotPath) throws RocksDBException { snapshotPath, null).rocksdb) { RocksDBStdSessions.createCheckpoint(rocksdb, snapshotLinkPath); } - LOG.debug("The snapshot data link path: {}", snapshotLinkPath); + LOG.debug("The snapshot {} has been hard linked to {}", + snapshotPath, snapshotLinkPath); return snapshotLinkPath; } @@ -438,6 +439,8 @@ private static Set mergeOldCFs(String path, List cfNames) } private static void createCheckpoint(RocksDB rocksdb, String targetPath) { + Path parentPath = Paths.get(targetPath).getParent().getFileName(); + assert parentPath.startsWith("snapshot") : targetPath; // https://github.com/facebook/rocksdb/wiki/Checkpoints try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { String tempPath = targetPath + "_temp"; @@ -1219,7 +1222,7 @@ private boolean filter(byte[] key) { */ assert this.keyEnd != null; if (this.match(Session.SCAN_LTE_END)) { - // Just compare the prefix, maybe there are excess tail + // Just compare the prefix, can be there are excess tail key = Arrays.copyOfRange(key, 0, this.keyEnd.length); return Bytes.compare(key, this.keyEnd) <= 0; } else { From 74c07645911acb52ac48f51ff28797ebd13ba406 Mon Sep 17 00:00:00 2001 From: liningrui Date: Fri, 26 Mar 2021 19:21:59 +0800 Subject: [PATCH 11/13] tiny improve Change-Id: Ie7367816e681438dd4bc82287d1b810d781845aa --- .../hugegraph/security/HugeSecurityManager.java | 14 +++++++------- .../backend/store/rocksdb/RocksDBStdSessions.java | 10 +++++----- .../backend/store/rocksdb/RocksDBStore.java | 4 ++-- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java index dc870d800c..de90486ecb 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/security/HugeSecurityManager.java @@ -95,7 +95,7 @@ public class HugeSecurityManager extends SecurityManager { ImmutableSet.of("execute") ); - private static final Map> ROCKSDB_SNAPSHOT = ImmutableMap.of( + private static final Map> BACKEND_SNAPSHOT = ImmutableMap.of( "com.baidu.hugegraph.backend.store.AbstractBackendStoreProvider", ImmutableSet.of("createSnapshot", "resumeSnapshot"), "com.baidu.hugegraph.backend.store.raft.RaftBackendStoreProvider", @@ -212,7 +212,7 @@ public void checkRead(FileDescriptor fd) { public void checkRead(String file) { if (callFromGremlin() && !callFromCaffeine() && !readGroovyInCurrentDir(file) && !callFromBackendHbase() && - !callFromBackendRocksDB() && !callFromRaft() && + !callFromSnapshot() && !callFromRaft() && !callFromSofaRpc()) { throw newSecurityException( "Not allowed to read file via Gremlin: %s", file); @@ -240,7 +240,7 @@ public void checkWrite(FileDescriptor fd) { @Override public void checkWrite(String file) { - if (callFromGremlin() && !callFromBackendRocksDB() && + if (callFromGremlin() && !callFromSnapshot() && !callFromRaft() && !callFromSofaRpc()) { throw newSecurityException("Not allowed to write file via Gremlin"); } @@ -249,7 +249,7 @@ public void checkWrite(String file) { @Override public void checkDelete(String file) { - if (callFromGremlin() && !callFromBackendRocksDB()) { + if (callFromGremlin() && !callFromSnapshot()) { throw newSecurityException( "Not allowed to delete file via Gremlin"); } @@ -332,7 +332,7 @@ public void checkPropertiesAccess() { public void checkPropertyAccess(String key) { if (!callFromAcceptClassLoaders() && callFromGremlin() && !WHITE_SYSTEM_PROPERTYS.contains(key) && !callFromBackendHbase() && - !callFromBackendRocksDB() && !callFromRaft() && + !callFromSnapshot() && !callFromRaft() && !callFromSofaRpc()) { throw newSecurityException( "Not allowed to access system property(%s) via Gremlin", key); @@ -453,8 +453,8 @@ private static boolean callFromBackendHbase() { return callFromWorkerWithClass(HBASE_CLASSES); } - private static boolean callFromBackendRocksDB() { - return callFromMethods(ROCKSDB_SNAPSHOT); + private static boolean callFromSnapshot() { + return callFromMethods(BACKEND_SNAPSHOT); } private static boolean callFromRaft() { diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 1224c65e78..b715975c83 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -129,7 +129,7 @@ public void open() throws Exception { @Override protected boolean opened() { - return this.rocksdb != null && this.rocksdb.rocksdb.isOwningHandle(); + return this.rocksdb != null && this.rocksdb.isOwningHandle(); } @Override @@ -293,15 +293,15 @@ public String buildSnapshotPath(String snapshotPrefix) { // Like: parent_path/snapshot_rocksdb-data/* Path snapshotPath = parentParentPath.resolve(snapshotPrefix + "_" + pureDataPath); - E.checkState(snapshotPath.toFile().exists(), - "The snapshot path '%s' doesn't exist", - snapshotPath); + E.checkArgument(snapshotPath.toFile().exists(), + "The snapshot path '%s' doesn't exist", + snapshotPath); return snapshotPath.toString(); } @Override public String hardLinkSnapshot(String snapshotPath) throws RocksDBException { - String snapshotLinkPath = this.dataPath + "_link"; + String snapshotLinkPath = this.dataPath + "_temp"; try (RocksDB rocksdb = openRocksDB(this.config, ImmutableList.of(), snapshotPath, null).rocksdb) { RocksDBStdSessions.createCheckpoint(rocksdb, snapshotLinkPath); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java index 579ad2fa6e..9cd31e530c 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStore.java @@ -629,7 +629,7 @@ public Set createSnapshot(String snapshotPrefix) { uniqueParents.add(snapshotPath.getParent().toString()); } - LOG.info("The store '{}' create snapshot successfully", this.store); + LOG.info("The store '{}' create snapshot successfully", this); return uniqueParents; } finally { readLock.unlock(); @@ -670,7 +670,7 @@ public void resumeSnapshot(String snapshotPrefix, boolean deleteSnapshot) { } } } - LOG.info("The store '{}' resume snapshot successfully", this.store); + LOG.info("The store '{}' resume snapshot successfully", this); } catch (RocksDBException | IOException e) { throw new BackendException("Failed to resume snapshot", e); } finally { From 6d06009e1e946397a14d794f935f1067e0556227 Mon Sep 17 00:00:00 2001 From: liningrui Date: Mon, 29 Mar 2021 14:34:23 +0800 Subject: [PATCH 12/13] tiny improve Change-Id: I70d1b3f70ac347f8e75a95d50654cf3f7f5536e8 --- .../hugegraph/backend/store/rocksdb/RocksDBStdSessions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index b715975c83..3e98cda349 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -440,7 +440,7 @@ private static Set mergeOldCFs(String path, List cfNames) private static void createCheckpoint(RocksDB rocksdb, String targetPath) { Path parentPath = Paths.get(targetPath).getParent().getFileName(); - assert parentPath.startsWith("snapshot") : targetPath; + assert parentPath.toString().startsWith("snapshot") : targetPath; // https://github.com/facebook/rocksdb/wiki/Checkpoints try (Checkpoint checkpoint = Checkpoint.create(rocksdb)) { String tempPath = targetPath + "_temp"; From 319ae663df5ce3fe6aa56aba1dce01d95157a928 Mon Sep 17 00:00:00 2001 From: liningrui Date: Wed, 31 Mar 2021 17:50:11 +0800 Subject: [PATCH 13/13] tiny improve Change-Id: I88b39003107652f6ff8eae071e88dc888dd21959 --- .../main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java | 4 ++-- .../baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java index eccb92c45c..76fd64e7b7 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/api/profile/GraphsAPI.java @@ -149,7 +149,7 @@ public Object createSnapshot(@Context GraphManager manager, HugeGraph g = graph(manager, name); g.createSnapshot(); - return ImmutableMap.of(name, "succeed"); + return ImmutableMap.of(name, "snapshot_created"); } @PUT @@ -163,7 +163,7 @@ public Object resumeSnapshot(@Context GraphManager manager, HugeGraph g = graph(manager, name); g.resumeSnapshot(); - return ImmutableMap.of(name, "succeed"); + return ImmutableMap.of(name, "snapshot_resumed"); } @PUT diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java index e3d8d77bd1..af9e35c4ee 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/StoreSnapshotFile.java @@ -86,7 +86,7 @@ public boolean load(SnapshotReader reader) { String jraftSnapshotPath = Paths.get(readerPath, SNAPSHOT_DIR) .toString(); try { - // decompress manifest and data directory + // Decompress manifest and data directory this.decompressSnapshot(readerPath, meta); this.doSnapshotLoad(); File tmp = new File(jraftSnapshotPath);